Series volume endpoint (#9704)

This changes the `label_volume` endpoint to the `series_volume`
endpoint. The new endpoint still returns volumes but now it does it for
the requested streams defined by the selector names passed rather than
individual labels. All relevant non-requested labels are aggregated into
the returned results

ex: Assume we have the following streams:
```
{cluster="prod", team="A", component="foo"}
{cluster="prod", team="B", component="foo"}
{cluster="dev", team="A", component="foo"}
{cluster="dev", team="B", component="foo"}
```

- requesting `{cluster="prod"}` returns one result for all streams
containing `{cluster="prod"}`
- requesting `{cluster=~".+"}` returns two results for the streams
containing `{cluster="prod"}` and `{cluster="dev"}`
- requesting `{cluster=~".+", team=".+"}` returns four results for the
streams containing:
```
{cluster="prod", team="A"}
{cluster="prod", team="B"}
{cluster="dev", team="A"}
{cluster="dev", team="B"}
```

---------

Co-authored-by: Trevor Whitney <trevorjwhitney@gmail.com>
pull/9670/head
Travis Patterson 3 years ago committed by GitHub
parent 093ec20b43
commit db97058a84
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      pkg/ingester/flush_test.go
  2. 22
      pkg/ingester/ingester.go
  3. 65
      pkg/ingester/ingester_test.go
  4. 39
      pkg/ingester/instance.go
  5. 49
      pkg/ingester/instance_test.go
  6. 8
      pkg/loghttp/query.go
  7. 18
      pkg/logproto/compat.go
  8. 78
      pkg/logproto/indexgateway.pb.go
  9. 2
      pkg/logproto/indexgateway.proto
  10. 500
      pkg/logproto/logproto.pb.go
  11. 10
      pkg/logproto/logproto.proto
  12. 8
      pkg/loki/modules.go
  13. 14
      pkg/querier/http.go
  14. 30
      pkg/querier/http_test.go
  15. 14
      pkg/querier/ingester_querier.go
  16. 22
      pkg/querier/ingester_querier_test.go
  17. 10
      pkg/querier/multi_tenant_querier.go
  18. 32
      pkg/querier/multi_tenant_querier_test.go
  19. 25
      pkg/querier/querier.go
  20. 16
      pkg/querier/querier_mock_test.go
  21. 10
      pkg/querier/querier_test.go
  22. 34
      pkg/querier/queryrange/codec.go
  23. 26
      pkg/querier/queryrange/codec_test.go
  24. 2
      pkg/querier/queryrange/extensions.go
  25. 256
      pkg/querier/queryrange/queryrange.pb.go
  26. 6
      pkg/querier/queryrange/queryrange.proto
  27. 16
      pkg/querier/queryrange/roundtrip.go
  28. 28
      pkg/querier/queryrange/roundtrip_test.go
  29. 6
      pkg/querier/queryrange/split_by_interval.go
  30. 26
      pkg/querier/queryrange/split_by_interval_test.go
  31. 33
      pkg/storage/async_store.go
  32. 34
      pkg/storage/async_store_test.go
  33. 10
      pkg/storage/stores/composite_store.go
  34. 17
      pkg/storage/stores/composite_store_entry.go
  35. 30
      pkg/storage/stores/composite_store_test.go
  36. 10
      pkg/storage/stores/index/index.go
  37. 102
      pkg/storage/stores/index/labelvolume/labelvolume.go
  38. 92
      pkg/storage/stores/index/seriesvolume/volume.go
  39. 8
      pkg/storage/stores/indexshipper/gatewayclient/gateway_client.go
  40. 7
      pkg/storage/stores/series/series_index_gateway_store.go
  41. 2
      pkg/storage/stores/series/series_index_store.go
  42. 8
      pkg/storage/stores/shipper/indexgateway/gateway.go
  43. 12
      pkg/storage/stores/shipper/indexgateway/gateway_test.go
  44. 4
      pkg/storage/stores/tsdb/head_manager.go
  45. 4
      pkg/storage/stores/tsdb/index.go
  46. 26
      pkg/storage/stores/tsdb/index_client.go
  47. 27
      pkg/storage/stores/tsdb/index_client_test.go
  48. 4
      pkg/storage/stores/tsdb/index_shipper_querier.go
  49. 4
      pkg/storage/stores/tsdb/lazy_index.go
  50. 4
      pkg/storage/stores/tsdb/multi_file_index.go
  51. 4
      pkg/storage/stores/tsdb/multitenant.go
  52. 56
      pkg/storage/stores/tsdb/single_file_index.go
  53. 87
      pkg/storage/stores/tsdb/single_file_index_test.go
  54. 2
      pkg/storage/util_test.go
  55. 2
      pkg/util/marshal/marshal.go

@ -351,8 +351,8 @@ func (s *testStore) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*la
return &stats.Stats{}, nil
}
func (s *testStore) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
return &logproto.LabelVolumeResponse{}, nil
func (s *testStore) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return &logproto.VolumeResponse{}, nil
}
func pushTestSamples(t *testing.T, ing logproto.PusherServer) map[string][]logproto.Stream {

@ -11,7 +11,7 @@ import (
"sync"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
@ -173,7 +173,7 @@ type ChunkStore interface {
GetChunkRefs(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([][]chunk.Chunk, []*fetcher.Fetcher, error)
GetSchemaConfigs() []config.PeriodConfig
Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*index_stats.Stats, error)
LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error)
SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error)
}
// Interface is an interface for the Ingester
@ -1140,7 +1140,7 @@ func (i *Ingester) GetStats(ctx context.Context, req *logproto.IndexStatsRequest
return &merged, nil
}
func (i *Ingester) GetLabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
func (i *Ingester) GetSeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
user, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
@ -1152,20 +1152,20 @@ func (i *Ingester) GetLabelVolume(ctx context.Context, req *logproto.LabelVolume
}
matchers, err := syntax.ParseMatchers(req.Matchers)
if err != nil && req.Matchers != labelvolume.MatchAny {
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
type f func() (*logproto.LabelVolumeResponse, error)
type f func() (*logproto.VolumeResponse, error)
jobs := []f{
f(func() (*logproto.LabelVolumeResponse, error) {
return instance.GetLabelVolume(ctx, req)
f(func() (*logproto.VolumeResponse, error) {
return instance.GetSeriesVolume(ctx, req)
}),
f(func() (*logproto.LabelVolumeResponse, error) {
return i.store.LabelVolume(ctx, user, req.From, req.Through, req.Limit, matchers...)
f(func() (*logproto.VolumeResponse, error) {
return i.store.SeriesVolume(ctx, user, req.From, req.Through, req.Limit, matchers...)
}),
}
resps := make([]*logproto.LabelVolumeResponse, len(jobs))
resps := make([]*logproto.VolumeResponse, len(jobs))
if err := concurrency.ForEachJob(
ctx,
@ -1180,7 +1180,7 @@ func (i *Ingester) GetLabelVolume(ctx context.Context, req *logproto.LabelVolume
return nil, err
}
merged := labelvolume.Merge(resps, req.Limit)
merged := seriesvolume.Merge(resps, req.Limit)
return merged, nil
}

@ -469,10 +469,10 @@ func (s *mockStore) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*la
}, nil
}
func (s *mockStore) LabelVolume(_ context.Context, _ string, _, _ model.Time, limit int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
return &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
func (s *mockStore) SeriesVolume(_ context.Context, _ string, _, _ model.Time, limit int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
Limit: limit,
}, nil
@ -1081,7 +1081,7 @@ func TestStats(t *testing.T) {
}, resp)
}
func TestLabelVolume(t *testing.T) {
func TestSeriesVolume(t *testing.T) {
ingesterConfig := defaultIngesterTestConfig(t)
limits, err := validation.NewOverrides(defaultLimitsTestConfig(), nil)
require.NoError(t, err)
@ -1092,20 +1092,51 @@ func TestLabelVolume(t *testing.T) {
i.instances["test"] = defaultInstance(t)
ctx := user.InjectOrgID(context.Background(), "test")
volumes, err := i.GetLabelVolume(ctx, &logproto.LabelVolumeRequest{
From: 0,
Through: 10000,
Matchers: "{}",
Limit: 4,
t.Run("matching a single label", func(t *testing.T) {
volumes, err := i.GetSeriesVolume(ctx, &logproto.VolumeRequest{
From: 0,
Through: 10000,
Matchers: `{log_stream=~"dispatcher|worker"}`,
Limit: 2,
})
require.NoError(t, err)
require.Equal(t, []logproto.Volume{
{Name: `{log_stream="dispatcher"}`, Value: "", Volume: 90},
{Name: `{log_stream="worker"}`, Value: "", Volume: 70},
}, volumes.Volumes)
})
t.Run("matching multiple labels, exact", func(t *testing.T) {
volumes, err := i.GetSeriesVolume(ctx, &logproto.VolumeRequest{
From: 0,
Through: 10000,
Matchers: `{log_stream=~"dispatcher|worker", host="agent"}`,
Limit: 2,
})
require.NoError(t, err)
require.Equal(t, []logproto.Volume{
{Name: `{host="agent", log_stream="dispatcher"}`, Value: "", Volume: 90},
{Name: `{host="agent", log_stream="worker"}`, Value: "", Volume: 70},
}, volumes.Volumes)
})
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{
{Name: "host", Value: "agent", Volume: 160},
{Name: "job", Value: "3", Volume: 160},
{Name: "log_stream", Value: "dispatcher", Volume: 90},
{Name: "log_stream", Value: "worker", Volume: 70},
}, volumes.Volumes)
t.Run("matching multiple labels, regex", func(t *testing.T) {
volumes, err := i.GetSeriesVolume(ctx, &logproto.VolumeRequest{
From: 0,
Through: 10000,
Matchers: `{log_stream=~"dispatcher|worker", host=~".+"}`,
Limit: 2,
})
require.NoError(t, err)
require.Equal(t, []logproto.Volume{
{Name: `{host="agent", log_stream="dispatcher"}`, Value: "", Volume: 90},
{Name: `{host="agent", log_stream="worker"}`, Value: "", Volume: 70},
}, volumes.Volumes)
})
}
type ingesterClient struct {

@ -8,7 +8,7 @@ import (
"syscall"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log/level"
"github.com/opentracing/opentracing-go"
@ -618,15 +618,28 @@ func (i *instance) GetStats(ctx context.Context, req *logproto.IndexStatsRequest
return res, nil
}
func (i *instance) GetLabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
func (i *instance) GetSeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
matchers, err := syntax.ParseMatchers(req.Matchers)
if err != nil && req.Matchers != labelvolume.MatchAny {
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
from, through := req.From.Time(), req.Through.Time()
matchAny := len(matchers) == 0
labelsToMatch := make(map[string]struct{})
for _, m := range matchers {
if m.Name == "" {
matchAny = true
continue
}
labelsToMatch[m.Name] = struct{}{}
}
volumes := make(map[string]map[string]uint64)
seriesNames := make(map[uint64]string)
seriesLabels := labels.Labels(make([]labels.Label, 0, len(labelsToMatch)))
from, through := req.From.Time(), req.Through.Time()
volumes := make(map[string]uint64)
if err = i.forMatchingStreams(ctx, from, matchers, nil, func(s *stream) error {
// Consider streams which overlap our time range
if shouldConsiderStream(s, from, through) {
@ -646,13 +659,21 @@ func (i *instance) GetLabelVolume(ctx context.Context, req *logproto.LabelVolume
}
}
seriesLabels = seriesLabels[:0]
for _, l := range s.labels {
if _, ok := volumes[l.Name]; !ok {
volumes[l.Name] = make(map[string]uint64)
if _, ok := labelsToMatch[l.Name]; matchAny || ok {
seriesLabels = append(seriesLabels, l)
}
volumes[l.Name][l.Value] += size
}
// If the labels are < 1k, this does not alloc
// https://github.com/prometheus/prometheus/pull/8025
hash := seriesLabels.Hash()
if _, ok := seriesNames[hash]; !ok {
seriesNames[hash] = seriesLabels.String()
}
volumes[seriesNames[hash]] += size
s.chunkMtx.RUnlock()
}
return nil
@ -660,7 +681,7 @@ func (i *instance) GetLabelVolume(ctx context.Context, req *logproto.LabelVolume
return nil, err
}
res := labelvolume.MapToLabelVolumeResponse(volumes, int(req.Limit))
res := seriesvolume.MapToSeriesVolumeResponse(volumes, int(req.Limit))
return res, nil
}

@ -838,44 +838,41 @@ func TestStreamShardingUsage(t *testing.T) {
})
}
func TestInstance_LabelVolume(t *testing.T) {
func TestInstance_SeriesVolume(t *testing.T) {
t.Run("no matchers", func(t *testing.T) {
instance := defaultInstance(t)
volumes, err := instance.GetLabelVolume(context.Background(), &logproto.LabelVolumeRequest{
volumes, err := instance.GetSeriesVolume(context.Background(), &logproto.VolumeRequest{
From: 0,
Through: 11000,
Matchers: "{}",
Limit: 3,
Limit: 2,
})
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{
{Name: "host", Value: "agent", Volume: 160},
{Name: "job", Value: "3", Volume: 160},
{Name: "log_stream", Value: "dispatcher", Volume: 90},
require.Equal(t, []logproto.Volume{
{Name: `{host="agent", job="3", log_stream="dispatcher"}`, Value: "", Volume: 90},
{Name: `{host="agent", job="3", log_stream="worker"}`, Value: "", Volume: 70},
}, volumes.Volumes)
})
t.Run("with matchers", func(t *testing.T) {
instance := defaultInstance(t)
volumes, err := instance.GetLabelVolume(context.Background(), &logproto.LabelVolumeRequest{
volumes, err := instance.GetSeriesVolume(context.Background(), &logproto.VolumeRequest{
From: 0,
Through: 11000,
Matchers: "{log_stream=\"dispatcher\"}",
Limit: 3,
Matchers: `{log_stream="dispatcher"}`,
Limit: 2,
})
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{
{Name: "host", Value: "agent", Volume: 90},
{Name: "job", Value: "3", Volume: 90},
{Name: "log_stream", Value: "dispatcher", Volume: 90},
require.Equal(t, []logproto.Volume{
{Name: `{log_stream="dispatcher"}`, Value: "", Volume: 90},
}, volumes.Volumes)
})
t.Run("excludes streams outside of time bounds", func(t *testing.T) {
instance := defaultInstance(t)
volumes, err := instance.GetLabelVolume(context.Background(), &logproto.LabelVolumeRequest{
volumes, err := instance.GetSeriesVolume(context.Background(), &logproto.VolumeRequest{
From: 5,
Through: 11,
Matchers: "{}",
@ -883,10 +880,24 @@ func TestInstance_LabelVolume(t *testing.T) {
})
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{
{Name: "host", Value: "agent", Volume: 71},
{Name: "job", Value: "3", Volume: 71},
{Name: "log_stream", Value: "dispatcher", Volume: 45},
require.Equal(t, []logproto.Volume{
{Name: `{host="agent", job="3", log_stream="dispatcher"}`, Value: "", Volume: 45},
{Name: `{host="agent", job="3", log_stream="worker"}`, Value: "", Volume: 26},
}, volumes.Volumes)
})
t.Run("enforces the limit", func(t *testing.T) {
instance := defaultInstance(t)
volumes, err := instance.GetSeriesVolume(context.Background(), &logproto.VolumeRequest{
From: 0,
Through: 11000,
Matchers: "{}",
Limit: 1,
})
require.NoError(t, err)
require.Equal(t, []logproto.Volume{
{Name: `{host="agent", job="3", log_stream="dispatcher"}`, Value: "", Volume: 90},
}, volumes.Volumes)
})
}

@ -7,7 +7,7 @@ import (
"time"
"unsafe"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/buger/jsonparser"
json "github.com/json-iterator/go"
@ -346,7 +346,7 @@ func ParseIndexStatsQuery(r *http.Request) (*RangeQuery, error) {
return ParseRangeQuery(r)
}
func ParseLabelVolumeQuery(r *http.Request) (*RangeQuery, error) {
func ParseSeriesVolumeQuery(r *http.Request) (*RangeQuery, error) {
err := labelVolumeLimit(r)
if err != nil {
return nil, err
@ -361,13 +361,13 @@ func ParseLabelVolumeQuery(r *http.Request) (*RangeQuery, error) {
}
func labelVolumeLimit(r *http.Request) error {
l, err := parseInt(r.Form.Get("limit"), labelvolume.DefaultLimit)
l, err := parseInt(r.Form.Get("limit"), seriesvolume.DefaultLimit)
if err != nil {
return err
}
if l == 0 {
r.Form.Set("limit", fmt.Sprint(labelvolume.DefaultLimit))
r.Form.Set("limit", fmt.Sprint(seriesvolume.DefaultLimit))
return nil
}

@ -317,31 +317,31 @@ func (m *IndexStatsRequest) LogToSpan(sp opentracing.Span) {
)
}
// Satisfy definitions.Request for LabelVolume
// Satisfy definitions.Request for Volume
// GetStart returns the start timestamp of the request in milliseconds.
func (m *LabelVolumeRequest) GetStart() int64 {
func (m *VolumeRequest) GetStart() int64 {
return int64(m.From)
}
// GetEnd returns the end timestamp of the request in milliseconds.
func (m *LabelVolumeRequest) GetEnd() int64 {
func (m *VolumeRequest) GetEnd() int64 {
return int64(m.Through)
}
// GetStep returns the step of the request in milliseconds.
func (m *LabelVolumeRequest) GetStep() int64 { return 0 }
func (m *VolumeRequest) GetStep() int64 { return 0 }
// GetQuery returns the query of the request.
func (m *LabelVolumeRequest) GetQuery() string {
func (m *VolumeRequest) GetQuery() string {
return m.Matchers
}
// GetCachingOptions returns the caching options.
func (m *LabelVolumeRequest) GetCachingOptions() (res definitions.CachingOptions) { return }
func (m *VolumeRequest) GetCachingOptions() (res definitions.CachingOptions) { return }
// WithStartEnd clone the current request with different start and end timestamp.
func (m *LabelVolumeRequest) WithStartEnd(startTime int64, endTime int64) definitions.Request {
func (m *VolumeRequest) WithStartEnd(startTime int64, endTime int64) definitions.Request {
clone := *m
clone.From = model.TimeFromUnixNano(startTime * int64(time.Millisecond))
clone.Through = model.TimeFromUnixNano(endTime * int64(time.Millisecond))
@ -349,14 +349,14 @@ func (m *LabelVolumeRequest) WithStartEnd(startTime int64, endTime int64) defini
}
// WithQuery clone the current request with a different query.
func (m *LabelVolumeRequest) WithQuery(query string) definitions.Request {
func (m *VolumeRequest) WithQuery(query string) definitions.Request {
clone := *m
clone.Matchers = query
return &clone
}
// LogToSpan writes information about this request to an OpenTracing span
func (m *LabelVolumeRequest) LogToSpan(sp opentracing.Span) {
func (m *VolumeRequest) LogToSpan(sp opentracing.Span) {
sp.LogFields(
otlog.String("query", m.GetQuery()),
otlog.String("start", timestamp.Time(m.GetStart()).String()),

@ -28,31 +28,31 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
func init() { proto.RegisterFile("pkg/logproto/indexgateway.proto", fileDescriptor_d27585148d0a52c8) }
var fileDescriptor_d27585148d0a52c8 = []byte{
// 374 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xc1, 0x4a, 0xe3, 0x40,
0x18, 0xc7, 0x33, 0xb0, 0x2c, 0xbb, 0xb3, 0x4b, 0x0f, 0x83, 0x50, 0x49, 0x75, 0x04, 0xf1, 0xa0,
0x97, 0x46, 0xf4, 0x0d, 0x14, 0x1a, 0x0a, 0x55, 0xb1, 0x82, 0x87, 0x1e, 0xc4, 0x49, 0xfd, 0x3a,
0x0d, 0x4d, 0x33, 0x31, 0x99, 0xa0, 0xbd, 0xf9, 0x08, 0x3e, 0x86, 0x8f, 0x22, 0x78, 0xe9, 0xb1,
0x47, 0x3b, 0xbd, 0x78, 0xec, 0x23, 0x48, 0x32, 0xa4, 0x9d, 0x96, 0x14, 0x3c, 0x65, 0xf2, 0xfb,
0xff, 0xe7, 0x37, 0x49, 0xbe, 0xe0, 0xbd, 0x68, 0xc0, 0x9d, 0x40, 0xf0, 0x28, 0x16, 0x52, 0x38,
0x7e, 0xf8, 0x00, 0xcf, 0x9c, 0x49, 0x78, 0x62, 0xa3, 0x7a, 0x8e, 0x48, 0xc5, 0x64, 0x91, 0x67,
0x6f, 0x71, 0xc1, 0x85, 0x6e, 0x67, 0x2b, 0xdd, 0xb2, 0x6b, 0x2b, 0x9a, 0x62, 0xa1, 0xc3, 0x93,
0x8f, 0x5f, 0xf8, 0x7f, 0x33, 0xb3, 0xb8, 0xda, 0x42, 0x9a, 0x18, 0x5f, 0xa7, 0x10, 0x8f, 0x72,
0x48, 0x6a, 0xf5, 0x45, 0x7f, 0x49, 0xdb, 0xf0, 0x98, 0x42, 0x22, 0xed, 0x9d, 0xf2, 0x30, 0x89,
0x44, 0x98, 0xc0, 0x31, 0x22, 0x2d, 0xfc, 0xcf, 0x05, 0x79, 0xde, 0x4f, 0xc3, 0x41, 0x1b, 0x7a,
0xc4, 0xa8, 0x1b, 0xb8, 0x90, 0xed, 0x6e, 0x48, 0xb5, 0x6d, 0xdf, 0x22, 0x0d, 0xfc, 0xd7, 0x05,
0x79, 0x03, 0xb1, 0x0f, 0x09, 0xb1, 0x57, 0xda, 0x1a, 0x16, 0xa6, 0x5a, 0x69, 0xb6, 0xf0, 0xdc,
0xe1, 0x6a, 0x8b, 0x79, 0x10, 0x5c, 0xb2, 0x21, 0x24, 0x0d, 0x11, 0x5f, 0x80, 0x8c, 0xfd, 0x6e,
0x76, 0x47, 0x0e, 0x97, 0x3b, 0x37, 0x54, 0x8a, 0x33, 0xaa, 0x6b, 0x4d, 0xc3, 0x7f, 0x8f, 0xb7,
0x73, 0x74, 0xcb, 0x82, 0x74, 0xfd, 0x80, 0xa3, 0xb5, 0x6d, 0x25, 0x9d, 0x1f, 0x9c, 0xe0, 0xe2,
0x3f, 0xd9, 0x8b, 0x49, 0x26, 0x13, 0x73, 0x40, 0xf9, 0xe7, 0xcf, 0x69, 0xc9, 0x80, 0xcc, 0x70,
0x21, 0xba, 0xc2, 0x15, 0x17, 0xa4, 0x7e, 0x12, 0x11, 0xa4, 0x43, 0x30, 0x67, 0x64, 0xe0, 0x92,
0x19, 0xad, 0xa4, 0x85, 0xf0, 0xac, 0x33, 0x9e, 0x52, 0x6b, 0x32, 0xa5, 0xd6, 0x7c, 0x4a, 0xd1,
0x8b, 0xa2, 0xe8, 0x4d, 0x51, 0xf4, 0xae, 0x28, 0x1a, 0x2b, 0x8a, 0x3e, 0x15, 0x45, 0x5f, 0x8a,
0x5a, 0x73, 0x45, 0xd1, 0xeb, 0x8c, 0x5a, 0xe3, 0x19, 0xb5, 0x26, 0x33, 0x6a, 0x75, 0x0e, 0xb8,
0x2f, 0xfb, 0xa9, 0x57, 0xef, 0x8a, 0xa1, 0xc3, 0x63, 0xd6, 0x63, 0x21, 0x73, 0x02, 0x31, 0xf0,
0x1d, 0xf3, 0xc7, 0xf5, 0x7e, 0xe7, 0x97, 0xd3, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe2, 0x3c,
0xaa, 0x2e, 0x16, 0x03, 0x00, 0x00,
// 371 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xc1, 0x4a, 0xfb, 0x30,
0x1c, 0xc7, 0x1b, 0xf8, 0xf3, 0x47, 0xa3, 0x28, 0x04, 0x61, 0xa3, 0xd3, 0x08, 0xe2, 0x41, 0x2f,
0xab, 0xe8, 0x1b, 0x28, 0xac, 0x0c, 0xa6, 0xe0, 0x84, 0x1d, 0x76, 0x10, 0xd3, 0xf9, 0x5b, 0x57,
0xd6, 0x35, 0xb5, 0x4d, 0xd1, 0xdd, 0x7c, 0x04, 0x1f, 0xc3, 0xe7, 0xf0, 0xe4, 0x71, 0xc7, 0x1d,
0x5d, 0x76, 0xf1, 0xb8, 0x47, 0x90, 0x36, 0x74, 0xcb, 0x66, 0x07, 0x9e, 0x9a, 0x7e, 0xbe, 0xdf,
0x7c, 0x7e, 0x34, 0x29, 0x3e, 0x0c, 0xfb, 0xae, 0xe5, 0x73, 0x37, 0x8c, 0xb8, 0xe0, 0x96, 0x17,
0x3c, 0xc2, 0x8b, 0xcb, 0x04, 0x3c, 0xb3, 0x61, 0x35, 0x43, 0x64, 0x47, 0x67, 0xa1, 0x63, 0xee,
0xb9, 0xdc, 0xe5, 0xaa, 0x9d, 0xae, 0x54, 0xcb, 0xac, 0x2c, 0x69, 0xf2, 0x85, 0x0a, 0xcf, 0x3f,
0xfe, 0xe1, 0xed, 0x7a, 0x6a, 0xb1, 0x95, 0x85, 0xd4, 0x31, 0xbe, 0x4d, 0x20, 0x1a, 0x66, 0x90,
0x54, 0xaa, 0xf3, 0xfe, 0x82, 0x36, 0xe1, 0x29, 0x81, 0x58, 0x98, 0xfb, 0xc5, 0x61, 0x1c, 0xf2,
0x20, 0x86, 0x33, 0x44, 0x1a, 0x78, 0xcb, 0x06, 0x71, 0xd5, 0x4b, 0x82, 0x7e, 0x13, 0xba, 0x44,
0xab, 0x6b, 0x38, 0x97, 0x1d, 0xac, 0x49, 0x95, 0xed, 0xc8, 0x20, 0x35, 0xbc, 0x69, 0x83, 0xb8,
0x83, 0xc8, 0x83, 0x98, 0x98, 0x4b, 0x6d, 0x05, 0x73, 0x53, 0xa5, 0x30, 0x9b, 0x7b, 0xee, 0x71,
0xa9, 0xc1, 0x1c, 0xf0, 0x6f, 0xd8, 0x00, 0xe2, 0x1a, 0x8f, 0xae, 0x41, 0x44, 0x5e, 0x27, 0x7d,
0x23, 0x27, 0x8b, 0x9d, 0x6b, 0x2a, 0xf9, 0x8c, 0xd2, 0x4a, 0x53, 0xf3, 0x3f, 0xe0, 0x72, 0x86,
0x5a, 0xcc, 0x4f, 0x56, 0x07, 0x9c, 0xae, 0x6c, 0x2b, 0xe8, 0xfc, 0x61, 0x82, 0x8d, 0x37, 0xd2,
0x0f, 0x13, 0x4c, 0xc4, 0xfa, 0x05, 0x65, 0xc7, 0x9f, 0xd1, 0x82, 0x0b, 0xd2, 0x43, 0xed, 0x48,
0x77, 0xe7, 0x27, 0xd4, 0xe2, 0x7e, 0x32, 0x00, 0xa2, 0x8d, 0x55, 0x24, 0x77, 0x95, 0x7f, 0x07,
0xb9, 0xe7, 0xb2, 0x3d, 0x9a, 0x50, 0x63, 0x3c, 0xa1, 0xc6, 0x6c, 0x42, 0xd1, 0xab, 0xa4, 0xe8,
0x5d, 0x52, 0xf4, 0x29, 0x29, 0x1a, 0x49, 0x8a, 0xbe, 0x24, 0x45, 0xdf, 0x92, 0x1a, 0x33, 0x49,
0xd1, 0xdb, 0x94, 0x1a, 0xa3, 0x29, 0x35, 0xc6, 0x53, 0x6a, 0xb4, 0x8f, 0x5d, 0x4f, 0xf4, 0x12,
0xa7, 0xda, 0xe1, 0x03, 0xcb, 0x8d, 0x58, 0x97, 0x05, 0xcc, 0xf2, 0x79, 0xdf, 0xb3, 0xf4, 0xff,
0xd5, 0xf9, 0x9f, 0x3d, 0x2e, 0x7e, 0x02, 0x00, 0x00, 0xff, 0xff, 0x79, 0x98, 0xe7, 0x4e, 0x0d,
0x03, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -80,7 +80,7 @@ type IndexGatewayClient interface {
GetStats(ctx context.Context, in *IndexStatsRequest, opts ...grpc.CallOption) (*IndexStatsResponse, error)
// Note: this MUST be the same as the variant defined in
// logproto.proto on the Querier service.
GetLabelVolume(ctx context.Context, in *LabelVolumeRequest, opts ...grpc.CallOption) (*LabelVolumeResponse, error)
GetSeriesVolume(ctx context.Context, in *VolumeRequest, opts ...grpc.CallOption) (*VolumeResponse, error)
}
type indexGatewayClient struct {
@ -168,9 +168,9 @@ func (c *indexGatewayClient) GetStats(ctx context.Context, in *IndexStatsRequest
return out, nil
}
func (c *indexGatewayClient) GetLabelVolume(ctx context.Context, in *LabelVolumeRequest, opts ...grpc.CallOption) (*LabelVolumeResponse, error) {
out := new(LabelVolumeResponse)
err := c.cc.Invoke(ctx, "/indexgatewaypb.IndexGateway/GetLabelVolume", in, out, opts...)
func (c *indexGatewayClient) GetSeriesVolume(ctx context.Context, in *VolumeRequest, opts ...grpc.CallOption) (*VolumeResponse, error) {
out := new(VolumeResponse)
err := c.cc.Invoke(ctx, "/indexgatewaypb.IndexGateway/GetSeriesVolume", in, out, opts...)
if err != nil {
return nil, err
}
@ -192,7 +192,7 @@ type IndexGatewayServer interface {
GetStats(context.Context, *IndexStatsRequest) (*IndexStatsResponse, error)
// Note: this MUST be the same as the variant defined in
// logproto.proto on the Querier service.
GetLabelVolume(context.Context, *LabelVolumeRequest) (*LabelVolumeResponse, error)
GetSeriesVolume(context.Context, *VolumeRequest) (*VolumeResponse, error)
}
// UnimplementedIndexGatewayServer can be embedded to have forward compatible implementations.
@ -217,8 +217,8 @@ func (*UnimplementedIndexGatewayServer) LabelValuesForMetricName(ctx context.Con
func (*UnimplementedIndexGatewayServer) GetStats(ctx context.Context, req *IndexStatsRequest) (*IndexStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetStats not implemented")
}
func (*UnimplementedIndexGatewayServer) GetLabelVolume(ctx context.Context, req *LabelVolumeRequest) (*LabelVolumeResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetLabelVolume not implemented")
func (*UnimplementedIndexGatewayServer) GetSeriesVolume(ctx context.Context, req *VolumeRequest) (*VolumeResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetSeriesVolume not implemented")
}
func RegisterIndexGatewayServer(s *grpc.Server, srv IndexGatewayServer) {
@ -336,20 +336,20 @@ func _IndexGateway_GetStats_Handler(srv interface{}, ctx context.Context, dec fu
return interceptor(ctx, in, info, handler)
}
func _IndexGateway_GetLabelVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LabelVolumeRequest)
func _IndexGateway_GetSeriesVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(VolumeRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(IndexGatewayServer).GetLabelVolume(ctx, in)
return srv.(IndexGatewayServer).GetSeriesVolume(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/indexgatewaypb.IndexGateway/GetLabelVolume",
FullMethod: "/indexgatewaypb.IndexGateway/GetSeriesVolume",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(IndexGatewayServer).GetLabelVolume(ctx, req.(*LabelVolumeRequest))
return srv.(IndexGatewayServer).GetSeriesVolume(ctx, req.(*VolumeRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -379,8 +379,8 @@ var _IndexGateway_serviceDesc = grpc.ServiceDesc{
Handler: _IndexGateway_GetStats_Handler,
},
{
MethodName: "GetLabelVolume",
Handler: _IndexGateway_GetLabelVolume_Handler,
MethodName: "GetSeriesVolume",
Handler: _IndexGateway_GetSeriesVolume_Handler,
},
},
Streams: []grpc.StreamDesc{

@ -25,5 +25,5 @@ service IndexGateway {
// Note: this MUST be the same as the variant defined in
// logproto.proto on the Querier service.
rpc GetLabelVolume(logproto.LabelVolumeRequest) returns (logproto.LabelVolumeResponse) {}
rpc GetSeriesVolume(logproto.VolumeRequest) returns (logproto.VolumeResponse) {}
}

@ -2285,24 +2285,24 @@ func (m *IndexStatsResponse) GetEntries() uint64 {
return 0
}
type LabelVolumeRequest struct {
type VolumeRequest struct {
From github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=from,proto3,customtype=github.com/prometheus/common/model.Time" json:"from"`
Through github_com_prometheus_common_model.Time `protobuf:"varint,2,opt,name=through,proto3,customtype=github.com/prometheus/common/model.Time" json:"through"`
Matchers string `protobuf:"bytes,3,opt,name=matchers,proto3" json:"matchers,omitempty"`
Limit int32 `protobuf:"varint,4,opt,name=limit,proto3" json:"limit,omitempty"`
}
func (m *LabelVolumeRequest) Reset() { *m = LabelVolumeRequest{} }
func (*LabelVolumeRequest) ProtoMessage() {}
func (*LabelVolumeRequest) Descriptor() ([]byte, []int) {
func (m *VolumeRequest) Reset() { *m = VolumeRequest{} }
func (*VolumeRequest) ProtoMessage() {}
func (*VolumeRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{42}
}
func (m *LabelVolumeRequest) XXX_Unmarshal(b []byte) error {
func (m *VolumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LabelVolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
func (m *VolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LabelVolumeRequest.Marshal(b, m, deterministic)
return xxx_messageInfo_VolumeRequest.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
@ -2312,48 +2312,48 @@ func (m *LabelVolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte,
return b[:n], nil
}
}
func (m *LabelVolumeRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_LabelVolumeRequest.Merge(m, src)
func (m *VolumeRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_VolumeRequest.Merge(m, src)
}
func (m *LabelVolumeRequest) XXX_Size() int {
func (m *VolumeRequest) XXX_Size() int {
return m.Size()
}
func (m *LabelVolumeRequest) XXX_DiscardUnknown() {
xxx_messageInfo_LabelVolumeRequest.DiscardUnknown(m)
func (m *VolumeRequest) XXX_DiscardUnknown() {
xxx_messageInfo_VolumeRequest.DiscardUnknown(m)
}
var xxx_messageInfo_LabelVolumeRequest proto.InternalMessageInfo
var xxx_messageInfo_VolumeRequest proto.InternalMessageInfo
func (m *LabelVolumeRequest) GetMatchers() string {
func (m *VolumeRequest) GetMatchers() string {
if m != nil {
return m.Matchers
}
return ""
}
func (m *LabelVolumeRequest) GetLimit() int32 {
func (m *VolumeRequest) GetLimit() int32 {
if m != nil {
return m.Limit
}
return 0
}
type LabelVolumeResponse struct {
Volumes []LabelVolume `protobuf:"bytes,1,rep,name=volumes,proto3" json:"volumes"`
Limit int32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"`
type VolumeResponse struct {
Volumes []Volume `protobuf:"bytes,1,rep,name=volumes,proto3" json:"volumes"`
Limit int32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"`
}
func (m *LabelVolumeResponse) Reset() { *m = LabelVolumeResponse{} }
func (*LabelVolumeResponse) ProtoMessage() {}
func (*LabelVolumeResponse) Descriptor() ([]byte, []int) {
func (m *VolumeResponse) Reset() { *m = VolumeResponse{} }
func (*VolumeResponse) ProtoMessage() {}
func (*VolumeResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{43}
}
func (m *LabelVolumeResponse) XXX_Unmarshal(b []byte) error {
func (m *VolumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LabelVolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
func (m *VolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LabelVolumeResponse.Marshal(b, m, deterministic)
return xxx_messageInfo_VolumeResponse.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
@ -2363,49 +2363,49 @@ func (m *LabelVolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte,
return b[:n], nil
}
}
func (m *LabelVolumeResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_LabelVolumeResponse.Merge(m, src)
func (m *VolumeResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_VolumeResponse.Merge(m, src)
}
func (m *LabelVolumeResponse) XXX_Size() int {
func (m *VolumeResponse) XXX_Size() int {
return m.Size()
}
func (m *LabelVolumeResponse) XXX_DiscardUnknown() {
xxx_messageInfo_LabelVolumeResponse.DiscardUnknown(m)
func (m *VolumeResponse) XXX_DiscardUnknown() {
xxx_messageInfo_VolumeResponse.DiscardUnknown(m)
}
var xxx_messageInfo_LabelVolumeResponse proto.InternalMessageInfo
var xxx_messageInfo_VolumeResponse proto.InternalMessageInfo
func (m *LabelVolumeResponse) GetVolumes() []LabelVolume {
func (m *VolumeResponse) GetVolumes() []Volume {
if m != nil {
return m.Volumes
}
return nil
}
func (m *LabelVolumeResponse) GetLimit() int32 {
func (m *VolumeResponse) GetLimit() int32 {
if m != nil {
return m.Limit
}
return 0
}
type LabelVolume struct {
type Volume struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name"`
Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value"`
Volume uint64 `protobuf:"varint,3,opt,name=volume,proto3" json:"volume"`
}
func (m *LabelVolume) Reset() { *m = LabelVolume{} }
func (*LabelVolume) ProtoMessage() {}
func (*LabelVolume) Descriptor() ([]byte, []int) {
func (m *Volume) Reset() { *m = Volume{} }
func (*Volume) ProtoMessage() {}
func (*Volume) Descriptor() ([]byte, []int) {
return fileDescriptor_c28a5f14f1f4c79a, []int{44}
}
func (m *LabelVolume) XXX_Unmarshal(b []byte) error {
func (m *Volume) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LabelVolume) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
func (m *Volume) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LabelVolume.Marshal(b, m, deterministic)
return xxx_messageInfo_Volume.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
@ -2415,33 +2415,33 @@ func (m *LabelVolume) XXX_Marshal(b []byte, deterministic bool) ([]byte, error)
return b[:n], nil
}
}
func (m *LabelVolume) XXX_Merge(src proto.Message) {
xxx_messageInfo_LabelVolume.Merge(m, src)
func (m *Volume) XXX_Merge(src proto.Message) {
xxx_messageInfo_Volume.Merge(m, src)
}
func (m *LabelVolume) XXX_Size() int {
func (m *Volume) XXX_Size() int {
return m.Size()
}
func (m *LabelVolume) XXX_DiscardUnknown() {
xxx_messageInfo_LabelVolume.DiscardUnknown(m)
func (m *Volume) XXX_DiscardUnknown() {
xxx_messageInfo_Volume.DiscardUnknown(m)
}
var xxx_messageInfo_LabelVolume proto.InternalMessageInfo
var xxx_messageInfo_Volume proto.InternalMessageInfo
func (m *LabelVolume) GetName() string {
func (m *Volume) GetName() string {
if m != nil {
return m.Name
}
return ""
}
func (m *LabelVolume) GetValue() string {
func (m *Volume) GetValue() string {
if m != nil {
return m.Value
}
return ""
}
func (m *LabelVolume) GetVolume() uint64 {
func (m *Volume) GetVolume() uint64 {
if m != nil {
return m.Volume
}
@ -2493,153 +2493,153 @@ func init() {
proto.RegisterType((*IndexQuery)(nil), "logproto.IndexQuery")
proto.RegisterType((*IndexStatsRequest)(nil), "logproto.IndexStatsRequest")
proto.RegisterType((*IndexStatsResponse)(nil), "logproto.IndexStatsResponse")
proto.RegisterType((*LabelVolumeRequest)(nil), "logproto.LabelVolumeRequest")
proto.RegisterType((*LabelVolumeResponse)(nil), "logproto.LabelVolumeResponse")
proto.RegisterType((*LabelVolume)(nil), "logproto.LabelVolume")
proto.RegisterType((*VolumeRequest)(nil), "logproto.VolumeRequest")
proto.RegisterType((*VolumeResponse)(nil), "logproto.VolumeResponse")
proto.RegisterType((*Volume)(nil), "logproto.Volume")
}
func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) }
var fileDescriptor_c28a5f14f1f4c79a = []byte{
// 2203 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0x4d, 0x6f, 0x1b, 0xc7,
0x95, 0xc3, 0x6f, 0x3e, 0x52, 0x12, 0x3d, 0x62, 0x6c, 0x96, 0xb1, 0x49, 0x79, 0x90, 0xda, 0x82,
0xed, 0x90, 0xb5, 0xd2, 0xa4, 0x8e, 0xdd, 0xb4, 0x30, 0xa5, 0x58, 0x91, 0x2d, 0x3b, 0xce, 0xc8,
0x75, 0x81, 0xa0, 0x81, 0xb1, 0x22, 0x87, 0x1f, 0x10, 0x97, 0x4b, 0xef, 0x2e, 0xe3, 0x08, 0xe8,
0xa1, 0xa7, 0xde, 0x02, 0xe4, 0x56, 0xf4, 0x52, 0xf4, 0x50, 0xa0, 0x45, 0x81, 0x5e, 0xfa, 0x03,
0xda, 0x1e, 0x0a, 0xd4, 0xbd, 0xb9, 0x97, 0x22, 0xe8, 0x81, 0xad, 0xe5, 0x4b, 0xa1, 0x53, 0x7e,
0x41, 0x51, 0xcc, 0xd7, 0xee, 0x2c, 0x45, 0x25, 0xa1, 0x6b, 0xa0, 0xf0, 0x85, 0x3b, 0xef, 0xcd,
0x9b, 0x99, 0xf7, 0xfd, 0xe6, 0x0d, 0xe1, 0xd5, 0xd1, 0x5e, 0xb7, 0x31, 0x70, 0xba, 0x23, 0xd7,
0xf1, 0x9d, 0x60, 0x50, 0x17, 0xbf, 0x38, 0xab, 0xe1, 0x4a, 0xa9, 0xeb, 0x74, 0x1d, 0x49, 0xc3,
0x47, 0x72, 0xbe, 0x52, 0xeb, 0x3a, 0x4e, 0x77, 0xc0, 0x1a, 0x02, 0xda, 0x1d, 0x77, 0x1a, 0x7e,
0xdf, 0x66, 0x9e, 0x6f, 0xd9, 0x23, 0x45, 0xb0, 0xa2, 0x76, 0x7f, 0x38, 0xb0, 0x9d, 0x36, 0x1b,
0x34, 0x3c, 0xdf, 0xf2, 0x3d, 0xf9, 0xab, 0x28, 0x96, 0x39, 0xc5, 0x68, 0xec, 0xf5, 0xc4, 0x8f,
0x44, 0x92, 0x12, 0xe0, 0x1d, 0xdf, 0x65, 0x96, 0x4d, 0x2d, 0x9f, 0x79, 0x94, 0x3d, 0x1c, 0x33,
0xcf, 0x27, 0xb7, 0x61, 0x39, 0x82, 0xf5, 0x46, 0xce, 0xd0, 0x63, 0xf8, 0x2d, 0xc8, 0x7b, 0x21,
0xba, 0x8c, 0x56, 0x12, 0xab, 0xf9, 0xb5, 0x52, 0x3d, 0x10, 0x25, 0x5c, 0x43, 0x4d, 0x42, 0xf2,
0x0b, 0x04, 0x10, 0xce, 0xe1, 0x2a, 0x80, 0x9c, 0x7d, 0xcf, 0xf2, 0x7a, 0x65, 0xb4, 0x82, 0x56,
0x93, 0xd4, 0xc0, 0xe0, 0x4b, 0x70, 0x22, 0x84, 0xee, 0x38, 0x3b, 0x3d, 0xcb, 0x6d, 0x97, 0xe3,
0x82, 0xec, 0xe8, 0x04, 0xc6, 0x90, 0x74, 0x2d, 0x9f, 0x95, 0x13, 0x2b, 0x68, 0x35, 0x41, 0xc5,
0x18, 0x9f, 0x84, 0xb4, 0xcf, 0x86, 0xd6, 0xd0, 0x2f, 0x27, 0x57, 0xd0, 0x6a, 0x8e, 0x2a, 0x88,
0xe3, 0xb9, 0xec, 0xcc, 0x2b, 0xa7, 0x56, 0xd0, 0xea, 0x02, 0x55, 0x10, 0xf9, 0x4b, 0x1c, 0x0a,
0x1f, 0x8c, 0x99, 0xbb, 0xaf, 0x14, 0x80, 0x2b, 0x90, 0xf5, 0xd8, 0x80, 0xb5, 0x7c, 0xc7, 0x15,
0x0c, 0xe6, 0x68, 0x00, 0xe3, 0x12, 0xa4, 0x06, 0x7d, 0xbb, 0xef, 0x0b, 0x96, 0x16, 0xa8, 0x04,
0xf0, 0x55, 0x48, 0x79, 0xbe, 0xe5, 0xfa, 0x82, 0x8f, 0xfc, 0x5a, 0xa5, 0x2e, 0x0d, 0x56, 0xd7,
0x06, 0xab, 0xdf, 0xd3, 0x06, 0x6b, 0x66, 0x1f, 0x4f, 0x6a, 0xb1, 0xcf, 0xfe, 0x59, 0x43, 0x54,
0x2e, 0xc1, 0x6f, 0x41, 0x82, 0x0d, 0xdb, 0x82, 0xd7, 0xaf, 0xbb, 0x92, 0x2f, 0xc0, 0x97, 0x21,
0xd7, 0xee, 0xbb, 0xac, 0xe5, 0xf7, 0x9d, 0xa1, 0x90, 0x68, 0x71, 0x6d, 0x39, 0xb4, 0xc6, 0x86,
0x9e, 0xa2, 0x21, 0x15, 0xbe, 0x04, 0x69, 0x8f, 0xab, 0xcd, 0x2b, 0x67, 0x56, 0x12, 0xab, 0xb9,
0x66, 0xe9, 0x70, 0x52, 0x2b, 0x4a, 0xcc, 0x25, 0xc7, 0xee, 0xfb, 0xcc, 0x1e, 0xf9, 0xfb, 0x54,
0xd1, 0xe0, 0x0b, 0x90, 0x69, 0xb3, 0x01, 0xe3, 0xc6, 0xce, 0x0a, 0x63, 0x17, 0x8d, 0xed, 0xc5,
0x04, 0xd5, 0x04, 0x37, 0x93, 0xd9, 0x74, 0x31, 0x43, 0xfe, 0x83, 0x00, 0xef, 0x58, 0xf6, 0x68,
0xc0, 0xbe, 0xb6, 0x3e, 0x03, 0xcd, 0xc5, 0x9f, 0x5b, 0x73, 0x89, 0x79, 0x35, 0x17, 0xaa, 0x21,
0x39, 0x9f, 0x1a, 0x52, 0x5f, 0xa1, 0x06, 0xb2, 0x0d, 0x69, 0x89, 0xfa, 0x2a, 0x1f, 0x0a, 0x65,
0x4e, 0x68, 0x69, 0x8a, 0xa1, 0x34, 0x09, 0xc1, 0x27, 0xf9, 0x25, 0x82, 0x05, 0xa5, 0x48, 0x15,
0x83, 0xbb, 0x90, 0x91, 0x31, 0xa0, 0xe3, 0xef, 0xd4, 0x74, 0xfc, 0x5d, 0x6f, 0x5b, 0x23, 0x9f,
0xb9, 0xcd, 0xc6, 0xe3, 0x49, 0x0d, 0xfd, 0x63, 0x52, 0x3b, 0xdf, 0xed, 0xfb, 0xbd, 0xf1, 0x6e,
0xbd, 0xe5, 0xd8, 0x8d, 0xae, 0x6b, 0x75, 0xac, 0xa1, 0xd5, 0x18, 0x38, 0x7b, 0xfd, 0x86, 0xce,
0x07, 0x3a, 0x6e, 0xf5, 0xc6, 0xf8, 0xa2, 0xe0, 0xce, 0xf7, 0x94, 0x45, 0x96, 0xea, 0x32, 0x8d,
0x6c, 0x0d, 0xbb, 0xcc, 0xe3, 0x3b, 0x27, 0xb9, 0x32, 0xa9, 0xa4, 0x21, 0x3f, 0x86, 0xe5, 0x88,
0xc1, 0x15, 0x9f, 0x57, 0x20, 0xed, 0x31, 0xb7, 0x1f, 0xa4, 0x09, 0x43, 0x65, 0x3b, 0x02, 0xdf,
0x5c, 0x54, 0xfc, 0xa5, 0x25, 0x4c, 0x15, 0xfd, 0x7c, 0xa7, 0xff, 0x19, 0x41, 0x61, 0xdb, 0xda,
0x65, 0x03, 0xed, 0x69, 0x18, 0x92, 0x43, 0xcb, 0x66, 0x4a, 0xe3, 0x62, 0xcc, 0xc3, 0xfe, 0x63,
0x6b, 0x30, 0x66, 0x72, 0xcb, 0x2c, 0x55, 0xd0, 0xbc, 0x31, 0x8b, 0x9e, 0x3b, 0x66, 0x51, 0xe8,
0x79, 0x25, 0x48, 0x3d, 0xe4, 0x8a, 0x12, 0xf1, 0x9a, 0xa3, 0x12, 0x20, 0xe7, 0x61, 0x41, 0x49,
0xa1, 0xd4, 0x17, 0xb2, 0xcc, 0xd5, 0x97, 0xd3, 0x2c, 0x13, 0x1b, 0xd2, 0x52, 0xdb, 0xf8, 0x35,
0xc8, 0x05, 0x35, 0x40, 0x48, 0x9b, 0x68, 0xa6, 0x0f, 0x27, 0xb5, 0xb8, 0xef, 0xd1, 0x70, 0x02,
0xd7, 0x20, 0x25, 0x56, 0x0a, 0xc9, 0x51, 0x33, 0x77, 0x38, 0xa9, 0x49, 0x04, 0x95, 0x1f, 0x7c,
0x1a, 0x92, 0x3d, 0x9e, 0x86, 0xb9, 0x0a, 0x92, 0xcd, 0xec, 0xe1, 0xa4, 0x26, 0x60, 0x2a, 0x7e,
0xc9, 0x26, 0x14, 0xb6, 0x59, 0xd7, 0x6a, 0xed, 0xab, 0x43, 0x4b, 0x7a, 0x3b, 0x7e, 0x20, 0xd2,
0x7b, 0x9c, 0x85, 0x42, 0x70, 0xe2, 0x03, 0xdb, 0x53, 0x4e, 0x9d, 0x0f, 0x70, 0xb7, 0x3d, 0xf2,
0x73, 0x04, 0xca, 0xce, 0x98, 0x40, 0x7a, 0xc0, 0x65, 0xf5, 0xa4, 0x8d, 0x9a, 0x70, 0x38, 0xa9,
0x29, 0x0c, 0x55, 0x5f, 0x7c, 0x0d, 0x32, 0x9e, 0x38, 0x91, 0x6f, 0x36, 0xed, 0x3e, 0x62, 0xa2,
0xb9, 0xc4, 0xdd, 0xe0, 0x70, 0x52, 0xd3, 0x84, 0x54, 0x0f, 0x70, 0x3d, 0x52, 0x5f, 0xa4, 0x60,
0x8b, 0x87, 0x93, 0x9a, 0x81, 0x35, 0xeb, 0x0d, 0xf9, 0x19, 0x82, 0xfc, 0x3d, 0xab, 0x1f, 0xb8,
0x50, 0x60, 0x22, 0x64, 0x98, 0x88, 0x87, 0x73, 0x9b, 0x0d, 0xac, 0xfd, 0x1b, 0x8e, 0x2b, 0xf6,
0x5c, 0xa0, 0x01, 0x1c, 0x96, 0x84, 0xe4, 0xcc, 0x92, 0x90, 0x9a, 0x3b, 0xb1, 0xdd, 0x4c, 0x66,
0xe3, 0xc5, 0x04, 0xf9, 0x1d, 0x82, 0x82, 0xe4, 0x4c, 0xb9, 0xc5, 0x8f, 0x20, 0x2d, 0x19, 0x17,
0xbc, 0x7d, 0x49, 0xf0, 0x5f, 0x9c, 0x27, 0xf0, 0xd5, 0x9e, 0xf8, 0xfb, 0xb0, 0xd8, 0x76, 0x9d,
0xd1, 0x88, 0xb5, 0x77, 0x54, 0x8a, 0x89, 0x4f, 0xa7, 0x98, 0x0d, 0x73, 0x9e, 0x4e, 0x91, 0x93,
0xbf, 0x22, 0x58, 0x50, 0xd1, 0xac, 0x74, 0x19, 0xe8, 0x00, 0x3d, 0x77, 0x72, 0x8f, 0xcf, 0x9b,
0xdc, 0x4f, 0x42, 0xba, 0xeb, 0x3a, 0xe3, 0x91, 0x57, 0x4e, 0xc8, 0xd8, 0x91, 0xd0, 0x7c, 0x49,
0x9f, 0xdc, 0x84, 0x45, 0x2d, 0xca, 0x31, 0x29, 0xad, 0x32, 0x9d, 0xd2, 0xb6, 0xda, 0x6c, 0xe8,
0xf7, 0x3b, 0xfd, 0x20, 0x49, 0x29, 0x7a, 0xf2, 0x29, 0x82, 0xe2, 0x34, 0x09, 0xfe, 0x9e, 0x11,
0x07, 0x7c, 0xbb, 0x73, 0xc7, 0x6f, 0x57, 0x17, 0xc9, 0xc1, 0x7b, 0x77, 0xe8, 0xbb, 0xfb, 0x3a,
0x46, 0x2a, 0x6f, 0x43, 0xde, 0x40, 0xf3, 0xe2, 0xb1, 0xc7, 0xb4, 0xcf, 0xf2, 0x61, 0x18, 0xac,
0x71, 0xe9, 0xc7, 0x02, 0xb8, 0x1a, 0xbf, 0x82, 0xb8, 0xc7, 0x2f, 0x44, 0x2c, 0x89, 0xaf, 0x40,
0xb2, 0xe3, 0x3a, 0xf6, 0x5c, 0x66, 0x12, 0x2b, 0xf0, 0xb7, 0x21, 0xee, 0x3b, 0x73, 0x19, 0x29,
0xee, 0x3b, 0xdc, 0x46, 0x4a, 0xf8, 0x84, 0xbc, 0xa1, 0x49, 0x88, 0xfc, 0x16, 0xc1, 0x12, 0x5f,
0x23, 0x35, 0xb0, 0xde, 0x1b, 0x0f, 0xf7, 0xf0, 0x2a, 0x14, 0xf9, 0x49, 0x0f, 0xfa, 0xaa, 0x02,
0x3c, 0xe8, 0xb7, 0x95, 0x98, 0x8b, 0x1c, 0xaf, 0x0b, 0xc3, 0x56, 0x1b, 0x9f, 0x82, 0xcc, 0xd8,
0x93, 0x04, 0x52, 0xe6, 0x34, 0x07, 0xb7, 0xda, 0xf8, 0xa2, 0x71, 0x1c, 0xd7, 0xb5, 0x71, 0x4d,
0x12, 0x3a, 0xbc, 0x6b, 0xf5, 0xdd, 0x20, 0xf9, 0x9c, 0x87, 0x74, 0x8b, 0x1f, 0x2c, 0xfd, 0x84,
0x57, 0xa0, 0x80, 0x58, 0x30, 0x44, 0xd5, 0x34, 0x79, 0x13, 0x72, 0xc1, 0xea, 0x99, 0x85, 0x67,
0xa6, 0x05, 0xc8, 0x35, 0x58, 0x92, 0x49, 0x75, 0xf6, 0xe2, 0xc2, 0xac, 0xc5, 0x05, 0xbd, 0xf8,
0x55, 0x48, 0x49, 0xad, 0x60, 0x48, 0xb6, 0x2d, 0xdf, 0xd2, 0x4b, 0xf8, 0x98, 0x94, 0xe1, 0xe4,
0x3d, 0xd7, 0x1a, 0x7a, 0x1d, 0xe6, 0x0a, 0xa2, 0xc0, 0x77, 0xc9, 0x2b, 0xb0, 0xcc, 0x13, 0x09,
0x73, 0xbd, 0x75, 0x67, 0x3c, 0xf4, 0xf5, 0x45, 0xff, 0x12, 0x94, 0xa2, 0x68, 0xe5, 0xea, 0x25,
0x48, 0xb5, 0x38, 0x42, 0xec, 0xbe, 0x40, 0x25, 0x40, 0x7e, 0x85, 0x00, 0x6f, 0x32, 0x5f, 0x6c,
0xbd, 0xb5, 0xe1, 0x19, 0x97, 0x3b, 0xdb, 0xf2, 0x5b, 0x3d, 0xe6, 0x7a, 0xfa, 0xa2, 0xa3, 0xe1,
0xff, 0xc7, 0xe5, 0x8e, 0x5c, 0x86, 0xe5, 0x08, 0x97, 0x4a, 0xa6, 0x0a, 0x64, 0x5b, 0x0a, 0xa7,
0x8a, 0x6a, 0x00, 0x93, 0xdf, 0xc7, 0x21, 0x2b, 0x6d, 0xcb, 0x3a, 0xf8, 0x32, 0xe4, 0x3b, 0xdc,
0xd7, 0xdc, 0x91, 0xdb, 0x57, 0x2a, 0x48, 0x36, 0x97, 0x0e, 0x27, 0x35, 0x13, 0x4d, 0x4d, 0x00,
0xbf, 0x3e, 0xe5, 0x78, 0xcd, 0xd2, 0xc1, 0xa4, 0x96, 0xfe, 0x01, 0x77, 0xbe, 0x0d, 0x5e, 0xde,
0x84, 0x1b, 0x6e, 0x04, 0xee, 0x78, 0x4b, 0x45, 0x9b, 0xb8, 0xe9, 0x35, 0xbf, 0xc3, 0xd9, 0x9f,
0xca, 0xd7, 0x23, 0xd7, 0xb1, 0x99, 0xdf, 0x63, 0x63, 0xaf, 0xd1, 0x72, 0x6c, 0xdb, 0x19, 0x36,
0x44, 0x5b, 0x27, 0x84, 0xe6, 0x35, 0x9a, 0x2f, 0x57, 0x01, 0x78, 0x0f, 0x32, 0x7e, 0xcf, 0x75,
0xc6, 0xdd, 0x9e, 0x28, 0x3f, 0x89, 0xe6, 0xd5, 0xf9, 0xf7, 0xd3, 0x3b, 0x50, 0x3d, 0xc0, 0x67,
0xb9, 0xb6, 0x58, 0x6b, 0xcf, 0x1b, 0xdb, 0xb2, 0x59, 0x6a, 0xa6, 0x0e, 0x27, 0x35, 0xf4, 0x3a,
0x0d, 0xd0, 0xe4, 0xd3, 0x38, 0xd4, 0x84, 0x0b, 0xdf, 0x17, 0x77, 0x93, 0x1b, 0x8e, 0x7b, 0x9b,
0xf9, 0x6e, 0xbf, 0x75, 0xc7, 0xb2, 0x99, 0xf6, 0x8d, 0x1a, 0xe4, 0x6d, 0x81, 0x7c, 0x60, 0x04,
0x07, 0xd8, 0x01, 0x1d, 0x3e, 0x03, 0x20, 0xc2, 0x4e, 0xce, 0xcb, 0x38, 0xc9, 0x09, 0x8c, 0x98,
0x5e, 0x8f, 0x68, 0xaa, 0x31, 0xa7, 0x64, 0x4a, 0x43, 0x5b, 0xd3, 0x1a, 0x9a, 0x7b, 0x9f, 0x40,
0x2d, 0xa6, 0xaf, 0xa7, 0xa2, 0xbe, 0x4e, 0xfe, 0x86, 0xa0, 0xba, 0xad, 0x39, 0x7f, 0x4e, 0x75,
0x68, 0x79, 0xe3, 0x2f, 0x48, 0xde, 0xc4, 0xff, 0x26, 0x2f, 0xf9, 0x93, 0x11, 0xf2, 0x94, 0x75,
0xb4, 0x1c, 0xeb, 0x46, 0xb9, 0x78, 0x11, 0x6c, 0xc6, 0x5f, 0xa0, 0x59, 0x12, 0x53, 0x66, 0x79,
0x27, 0x4c, 0x07, 0x42, 0x02, 0x95, 0x0e, 0xce, 0x41, 0xd2, 0x65, 0x1d, 0x5d, 0x7c, 0xf1, 0x74,
0x8e, 0x67, 0x1d, 0x2a, 0xe6, 0xc9, 0x1f, 0x10, 0x14, 0x37, 0x99, 0x1f, 0xbd, 0xd6, 0xbc, 0x4c,
0xf2, 0xbf, 0x07, 0x27, 0x0c, 0xfe, 0x95, 0xf4, 0x6f, 0x4c, 0xdd, 0x65, 0x5e, 0x09, 0xe5, 0xdf,
0x1a, 0xb6, 0xd9, 0x27, 0xaa, 0x47, 0x8b, 0x5e, 0x63, 0xee, 0x42, 0xde, 0x98, 0xc4, 0xd7, 0xa7,
0x2e, 0x30, 0xb3, 0x8a, 0x6a, 0xb3, 0xa4, 0x64, 0x92, 0x5d, 0x9a, 0xba, 0x9e, 0x06, 0xe5, 0x7e,
0x07, 0xb0, 0x68, 0x1b, 0xc5, 0xb6, 0x66, 0xa6, 0x16, 0xd8, 0x5b, 0xc1, 0x7d, 0x26, 0x80, 0xf1,
0x59, 0x48, 0xba, 0xce, 0x23, 0x7d, 0x33, 0x5d, 0x08, 0x8f, 0xa4, 0xce, 0x23, 0x2a, 0xa6, 0xc8,
0x35, 0x48, 0x50, 0xe7, 0x11, 0xae, 0x02, 0xb8, 0xd6, 0xb0, 0xcb, 0xee, 0x07, 0x0d, 0x4b, 0x81,
0x1a, 0x98, 0x63, 0xea, 0xeb, 0x3a, 0x9c, 0x30, 0x39, 0x92, 0xe6, 0xae, 0x43, 0x86, 0x23, 0xfb,
0xb3, 0x1e, 0xbd, 0x04, 0xa1, 0xec, 0x7d, 0x35, 0x11, 0xf7, 0x19, 0x08, 0xf1, 0xf8, 0x34, 0xe4,
0x7c, 0x6b, 0x77, 0xc0, 0xee, 0x84, 0x31, 0x1f, 0x22, 0xf8, 0x2c, 0xef, 0xb5, 0xee, 0x1b, 0x17,
0x85, 0x10, 0x81, 0x2f, 0x40, 0x31, 0xe4, 0xf9, 0xae, 0xcb, 0x3a, 0xfd, 0x4f, 0x84, 0x85, 0x0b,
0xf4, 0x08, 0x1e, 0xaf, 0xc2, 0x52, 0x88, 0xdb, 0x11, 0x65, 0x37, 0x29, 0x48, 0xa7, 0xd1, 0x5c,
0x37, 0x42, 0xdc, 0x77, 0x1f, 0x8e, 0xad, 0x81, 0x48, 0x64, 0x05, 0x6a, 0x60, 0xc8, 0x1f, 0x11,
0x9c, 0x90, 0xa6, 0xe6, 0x5d, 0xf6, 0xcb, 0xe8, 0xf5, 0xbf, 0x46, 0x80, 0x4d, 0x09, 0x94, 0x6b,
0x7d, 0xd3, 0x7c, 0x3e, 0xe1, 0x75, 0x3d, 0x2f, 0x5a, 0x48, 0x89, 0x0a, 0x5f, 0x40, 0x48, 0x70,
0x05, 0x14, 0xef, 0x8e, 0xb2, 0x47, 0x95, 0x18, 0x7d, 0xfb, 0xe3, 0xad, 0xf5, 0xee, 0xbe, 0xcf,
0x3c, 0xd5, 0x61, 0x8a, 0xd6, 0x5a, 0x20, 0xa8, 0xfc, 0xf0, 0xb3, 0xd8, 0xd0, 0x17, 0x5e, 0x93,
0x0c, 0xcf, 0x52, 0x28, 0xaa, 0x07, 0xe4, 0xef, 0x08, 0xb0, 0x2c, 0xa3, 0xce, 0x60, 0x1c, 0x96,
0x8a, 0x97, 0x48, 0xd9, 0xd1, 0xfe, 0x37, 0xa5, 0xfa, 0x5f, 0xb2, 0x0b, 0xcb, 0x11, 0xb9, 0x94,
0x09, 0xde, 0x84, 0xcc, 0xc7, 0x02, 0x33, 0x23, 0xf7, 0x18, 0xf4, 0x2a, 0xf7, 0x68, 0xda, 0xe8,
0xb3, 0x6b, 0x70, 0xc6, 0x48, 0x35, 0x41, 0x72, 0x0d, 0x3e, 0x6d, 0x5e, 0xc2, 0xe5, 0x6b, 0x06,
0x87, 0xd5, 0x8d, 0x3a, 0xf2, 0x18, 0x92, 0x9b, 0xf1, 0x18, 0x42, 0x20, 0x2d, 0x8f, 0x53, 0x36,
0x15, 0x66, 0x97, 0x18, 0xaa, 0xbe, 0x17, 0xce, 0x41, 0x2e, 0x78, 0x59, 0xc5, 0x79, 0xc8, 0xdc,
0x78, 0x9f, 0xfe, 0xf0, 0x3a, 0xdd, 0x28, 0xc6, 0x70, 0x01, 0xb2, 0xcd, 0xeb, 0xeb, 0xb7, 0x04,
0x84, 0xd6, 0x7e, 0x9a, 0xd2, 0x49, 0xc3, 0xc5, 0xdf, 0x85, 0x94, 0xcc, 0x04, 0x27, 0x43, 0x51,
0xcd, 0xf7, 0xd1, 0xca, 0xa9, 0x23, 0x78, 0x75, 0x6f, 0x8f, 0x7d, 0x0b, 0xe1, 0x3b, 0x90, 0x17,
0x48, 0xf5, 0x06, 0x73, 0x7a, 0xfa, 0x29, 0x24, 0xb2, 0xd3, 0x99, 0x63, 0x66, 0x8d, 0xfd, 0xae,
0x42, 0x4a, 0xe8, 0xcc, 0xe4, 0xc6, 0x7c, 0x43, 0x33, 0xb9, 0x89, 0xbc, 0x4a, 0x91, 0x18, 0x7e,
0x1b, 0x92, 0xbc, 0x61, 0xc0, 0x86, 0xcd, 0x8c, 0xa7, 0x93, 0xca, 0xc9, 0x69, 0xb4, 0x71, 0xec,
0x3b, 0xc1, 0x0b, 0xd0, 0xa9, 0xe9, 0x4e, 0x57, 0x2f, 0x2f, 0x1f, 0x9d, 0x08, 0x4e, 0x7e, 0x5f,
0x3e, 0x85, 0xe8, 0x56, 0x05, 0x9f, 0x89, 0x1e, 0x35, 0xd5, 0xd9, 0x54, 0xaa, 0xc7, 0x4d, 0x07,
0x1b, 0x6e, 0x43, 0xde, 0x68, 0x13, 0x4c, 0xb5, 0x1e, 0xed, 0x71, 0x4c, 0xb5, 0xce, 0xe8, 0x2d,
0x48, 0x0c, 0x6f, 0x42, 0x96, 0x57, 0x59, 0x9e, 0x6c, 0xf0, 0xab, 0xd3, 0xc5, 0xd4, 0x48, 0xa2,
0x95, 0xd3, 0xb3, 0x27, 0x0d, 0x39, 0x17, 0x37, 0x99, 0x1f, 0x71, 0xea, 0x99, 0xf1, 0x31, 0x83,
0xb3, 0x19, 0xd1, 0x46, 0x62, 0x6b, 0x1f, 0x41, 0x56, 0xb7, 0xc8, 0xf8, 0x03, 0x58, 0x8c, 0x36,
0x88, 0xf8, 0x1b, 0x86, 0x9e, 0xa2, 0x7d, 0x77, 0x65, 0xc5, 0x98, 0x9a, 0xdd, 0x55, 0xc6, 0x56,
0xd1, 0xda, 0x47, 0xfa, 0xbf, 0x9d, 0x0d, 0xcb, 0xb7, 0x14, 0xf7, 0xc6, 0x9f, 0x47, 0x11, 0x77,
0x3d, 0xf2, 0x4f, 0x53, 0xc4, 0x5d, 0x8f, 0xfe, 0xe3, 0x44, 0x62, 0xcd, 0x0f, 0x9f, 0x3c, 0xad,
0xc6, 0x3e, 0x7f, 0x5a, 0x8d, 0x7d, 0xf1, 0xb4, 0x8a, 0x7e, 0x72, 0x50, 0x45, 0xbf, 0x39, 0xa8,
0xa2, 0xc7, 0x07, 0x55, 0xf4, 0xe4, 0xa0, 0x8a, 0xfe, 0x75, 0x50, 0x45, 0xff, 0x3e, 0xa8, 0xc6,
0xbe, 0x38, 0xa8, 0xa2, 0xcf, 0x9e, 0x55, 0x63, 0x4f, 0x9e, 0x55, 0x63, 0x9f, 0x3f, 0xab, 0xc6,
0x3e, 0x7c, 0xed, 0xcb, 0x9e, 0xbe, 0xf4, 0x89, 0xbb, 0x69, 0xf1, 0x79, 0xe3, 0xbf, 0x01, 0x00,
0x00, 0xff, 0xff, 0xa0, 0x20, 0x68, 0x48, 0x9a, 0x1b, 0x00, 0x00,
// 2205 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0x4d, 0x8f, 0x1b, 0x49,
0xd5, 0x65, 0xb7, 0xbf, 0x9e, 0xed, 0x19, 0xa7, 0xc6, 0x9b, 0x18, 0x27, 0xb1, 0x27, 0xad, 0x25,
0x19, 0x25, 0x59, 0x7b, 0x33, 0x0b, 0x4b, 0x36, 0x61, 0x41, 0xf1, 0xcc, 0x26, 0x3b, 0xf9, 0xde,
0x9a, 0x10, 0xd0, 0x8a, 0x55, 0xd4, 0x63, 0x97, 0x3f, 0x34, 0x6e, 0xb7, 0xd3, 0xdd, 0xde, 0xec,
0x48, 0x1c, 0xf8, 0x03, 0x2b, 0xed, 0x0d, 0x71, 0x41, 0x1c, 0x90, 0x40, 0x48, 0x5c, 0xf8, 0x01,
0xc0, 0x01, 0x89, 0x70, 0x0b, 0xb7, 0x15, 0x07, 0x43, 0x26, 0x17, 0x34, 0xa7, 0xfd, 0x03, 0x20,
0x54, 0x5f, 0xdd, 0xd5, 0x3d, 0xce, 0xee, 0x3a, 0x44, 0x42, 0xb9, 0x8c, 0xfb, 0xbd, 0x7a, 0xf5,
0xea, 0x7d, 0xbf, 0x7a, 0x35, 0x70, 0x7c, 0xb2, 0xdb, 0x6f, 0x8d, 0x9c, 0xfe, 0xc4, 0x75, 0x7c,
0x27, 0xf8, 0x68, 0xf2, 0xbf, 0x38, 0xa7, 0xe0, 0x5a, 0xa5, 0xef, 0xf4, 0x1d, 0x41, 0xc3, 0xbe,
0xc4, 0x7a, 0xad, 0xd1, 0x77, 0x9c, 0xfe, 0x88, 0xb6, 0x38, 0xb4, 0x33, 0xed, 0xb5, 0xfc, 0xa1,
0x4d, 0x3d, 0xdf, 0xb2, 0x27, 0x92, 0x60, 0x55, 0x72, 0x7f, 0x38, 0xb2, 0x9d, 0x2e, 0x1d, 0xb5,
0x3c, 0xdf, 0xf2, 0x3d, 0xf1, 0x57, 0x52, 0xac, 0x30, 0x8a, 0xc9, 0xd4, 0x1b, 0xf0, 0x3f, 0x02,
0x69, 0x56, 0x00, 0x6f, 0xfb, 0x2e, 0xb5, 0x6c, 0x62, 0xf9, 0xd4, 0x23, 0xf4, 0xe1, 0x94, 0x7a,
0xbe, 0x79, 0x0b, 0x56, 0x22, 0x58, 0x6f, 0xe2, 0x8c, 0x3d, 0x8a, 0xdf, 0x86, 0x82, 0x17, 0xa2,
0xab, 0x68, 0x35, 0xb5, 0x56, 0x58, 0xaf, 0x34, 0x03, 0x55, 0xc2, 0x3d, 0x44, 0x27, 0x34, 0x7f,
0x81, 0x00, 0xc2, 0x35, 0x5c, 0x07, 0x10, 0xab, 0xef, 0x5b, 0xde, 0xa0, 0x8a, 0x56, 0xd1, 0x9a,
0x41, 0x34, 0x0c, 0x3e, 0x0f, 0x47, 0x42, 0xe8, 0xb6, 0xb3, 0x3d, 0xb0, 0xdc, 0x6e, 0x35, 0xc9,
0xc9, 0x0e, 0x2f, 0x60, 0x0c, 0x86, 0x6b, 0xf9, 0xb4, 0x9a, 0x5a, 0x45, 0x6b, 0x29, 0xc2, 0xbf,
0xf1, 0x51, 0xc8, 0xf8, 0x74, 0x6c, 0x8d, 0xfd, 0xaa, 0xb1, 0x8a, 0xd6, 0xf2, 0x44, 0x42, 0x0c,
0xcf, 0x74, 0xa7, 0x5e, 0x35, 0xbd, 0x8a, 0xd6, 0x4a, 0x44, 0x42, 0xe6, 0x5f, 0x92, 0x50, 0xfc,
0x60, 0x4a, 0xdd, 0x3d, 0x69, 0x00, 0x5c, 0x83, 0x9c, 0x47, 0x47, 0xb4, 0xe3, 0x3b, 0x2e, 0x17,
0x30, 0x4f, 0x02, 0x18, 0x57, 0x20, 0x3d, 0x1a, 0xda, 0x43, 0x9f, 0x8b, 0x54, 0x22, 0x02, 0xc0,
0x97, 0x20, 0xed, 0xf9, 0x96, 0xeb, 0x73, 0x39, 0x0a, 0xeb, 0xb5, 0xa6, 0x70, 0x58, 0x53, 0x39,
0xac, 0x79, 0x4f, 0x39, 0xac, 0x9d, 0x7b, 0x3c, 0x6b, 0x24, 0x3e, 0xfb, 0x47, 0x03, 0x11, 0xb1,
0x05, 0xbf, 0x0d, 0x29, 0x3a, 0xee, 0x72, 0x59, 0xbf, 0xee, 0x4e, 0xb6, 0x01, 0x5f, 0x80, 0x7c,
0x77, 0xe8, 0xd2, 0x8e, 0x3f, 0x74, 0xc6, 0x5c, 0xa3, 0xa5, 0xf5, 0x95, 0xd0, 0x1b, 0x9b, 0x6a,
0x89, 0x84, 0x54, 0xf8, 0x3c, 0x64, 0x3c, 0x66, 0x36, 0xaf, 0x9a, 0x5d, 0x4d, 0xad, 0xe5, 0xdb,
0x95, 0x83, 0x59, 0xa3, 0x2c, 0x30, 0xe7, 0x1d, 0x7b, 0xe8, 0x53, 0x7b, 0xe2, 0xef, 0x11, 0x49,
0x83, 0xcf, 0x42, 0xb6, 0x4b, 0x47, 0x94, 0x39, 0x3b, 0xc7, 0x9d, 0x5d, 0xd6, 0xd8, 0xf3, 0x05,
0xa2, 0x08, 0xae, 0x1b, 0xb9, 0x4c, 0x39, 0x6b, 0xfe, 0x07, 0x01, 0xde, 0xb6, 0xec, 0xc9, 0x88,
0x7e, 0x6d, 0x7b, 0x06, 0x96, 0x4b, 0xbe, 0xb0, 0xe5, 0x52, 0x8b, 0x5a, 0x2e, 0x34, 0x83, 0xb1,
0x98, 0x19, 0xd2, 0x5f, 0x61, 0x06, 0xf3, 0x26, 0x64, 0x04, 0xea, 0xab, 0x62, 0x28, 0xd4, 0x39,
0xa5, 0xb4, 0x29, 0x87, 0xda, 0xa4, 0xb8, 0x9c, 0xe6, 0x2f, 0x11, 0x94, 0xa4, 0x21, 0x65, 0x0e,
0xee, 0x40, 0x56, 0xe4, 0x80, 0xca, 0xbf, 0x63, 0xf1, 0xfc, 0xbb, 0xd2, 0xb5, 0x26, 0x3e, 0x75,
0xdb, 0xad, 0xc7, 0xb3, 0x06, 0xfa, 0xfb, 0xac, 0x71, 0xa6, 0x3f, 0xf4, 0x07, 0xd3, 0x9d, 0x66,
0xc7, 0xb1, 0x5b, 0x7d, 0xd7, 0xea, 0x59, 0x63, 0xab, 0x35, 0x72, 0x76, 0x87, 0x2d, 0x55, 0x0f,
0x54, 0xde, 0x2a, 0xc6, 0xf8, 0x1c, 0x97, 0xce, 0xf7, 0xa4, 0x47, 0x96, 0x9b, 0xa2, 0x8c, 0x6c,
0x8d, 0xfb, 0xd4, 0x63, 0x9c, 0x0d, 0x66, 0x4c, 0x22, 0x68, 0xcc, 0x9f, 0xc0, 0x4a, 0xc4, 0xe1,
0x52, 0xce, 0x8b, 0x90, 0xf1, 0xa8, 0x3b, 0x0c, 0xca, 0x84, 0x66, 0xb2, 0x6d, 0x8e, 0x6f, 0x2f,
0x49, 0xf9, 0x32, 0x02, 0x26, 0x92, 0x7e, 0xb1, 0xd3, 0xff, 0x8c, 0xa0, 0x78, 0xd3, 0xda, 0xa1,
0x23, 0x15, 0x69, 0x18, 0x8c, 0xb1, 0x65, 0x53, 0x69, 0x71, 0xfe, 0xcd, 0xd2, 0xfe, 0x63, 0x6b,
0x34, 0xa5, 0x82, 0x65, 0x8e, 0x48, 0x68, 0xd1, 0x9c, 0x45, 0x2f, 0x9c, 0xb3, 0x28, 0x8c, 0xbc,
0x0a, 0xa4, 0x1f, 0x32, 0x43, 0xf1, 0x7c, 0xcd, 0x13, 0x01, 0x98, 0x67, 0xa0, 0x24, 0xb5, 0x90,
0xe6, 0x0b, 0x45, 0x66, 0xe6, 0xcb, 0x2b, 0x91, 0x4d, 0x1b, 0x32, 0xc2, 0xda, 0xf8, 0x75, 0xc8,
0x07, 0x3d, 0x80, 0x6b, 0x9b, 0x6a, 0x67, 0x0e, 0x66, 0x8d, 0xa4, 0xef, 0x91, 0x70, 0x01, 0x37,
0x20, 0xcd, 0x77, 0x72, 0xcd, 0x51, 0x3b, 0x7f, 0x30, 0x6b, 0x08, 0x04, 0x11, 0x3f, 0xf8, 0x04,
0x18, 0x03, 0x56, 0x86, 0x99, 0x09, 0x8c, 0x76, 0xee, 0x60, 0xd6, 0xe0, 0x30, 0xe1, 0x7f, 0xcd,
0x6b, 0x50, 0xbc, 0x49, 0xfb, 0x56, 0x67, 0x4f, 0x1e, 0x5a, 0x51, 0xec, 0xd8, 0x81, 0x48, 0xf1,
0x38, 0x05, 0xc5, 0xe0, 0xc4, 0x07, 0xb6, 0x27, 0x83, 0xba, 0x10, 0xe0, 0x6e, 0x79, 0xe6, 0xcf,
0x11, 0x48, 0x3f, 0x63, 0x13, 0x32, 0x23, 0xa6, 0xab, 0x27, 0x7c, 0xd4, 0x86, 0x83, 0x59, 0x43,
0x62, 0x88, 0xfc, 0xc5, 0x97, 0x21, 0xeb, 0xf1, 0x13, 0x19, 0xb3, 0x78, 0xf8, 0xf0, 0x85, 0xf6,
0x32, 0x0b, 0x83, 0x83, 0x59, 0x43, 0x11, 0x12, 0xf5, 0x81, 0x9b, 0x91, 0xfe, 0x22, 0x14, 0x5b,
0x3a, 0x98, 0x35, 0x34, 0xac, 0xde, 0x6f, 0xcc, 0x9f, 0x21, 0x28, 0xdc, 0xb3, 0x86, 0x41, 0x08,
0x05, 0x2e, 0x42, 0x9a, 0x8b, 0x58, 0x3a, 0x77, 0xe9, 0xc8, 0xda, 0xbb, 0xea, 0xb8, 0x9c, 0x67,
0x89, 0x04, 0x70, 0xd8, 0x12, 0x8c, 0xb9, 0x2d, 0x21, 0xbd, 0x70, 0x61, 0xbb, 0x6e, 0xe4, 0x92,
0xe5, 0x94, 0xf9, 0x3b, 0x04, 0x45, 0x21, 0x99, 0x0c, 0x8b, 0x1f, 0x43, 0x46, 0x08, 0xce, 0x65,
0xfb, 0x92, 0xe4, 0x3f, 0xb7, 0x48, 0xe2, 0x4b, 0x9e, 0xf8, 0xfb, 0xb0, 0xd4, 0x75, 0x9d, 0xc9,
0x84, 0x76, 0xb7, 0x65, 0x89, 0x49, 0xc6, 0x4b, 0xcc, 0xa6, 0xbe, 0x4e, 0x62, 0xe4, 0xe6, 0x5f,
0x11, 0x94, 0x64, 0x36, 0x4b, 0x5b, 0x06, 0x36, 0x40, 0x2f, 0x5c, 0xdc, 0x93, 0x8b, 0x16, 0xf7,
0xa3, 0x90, 0xe9, 0xbb, 0xce, 0x74, 0xe2, 0x55, 0x53, 0x22, 0x77, 0x04, 0xb4, 0x58, 0xd1, 0x37,
0xaf, 0xc3, 0x92, 0x52, 0xe5, 0x39, 0x25, 0xad, 0x16, 0x2f, 0x69, 0x5b, 0x5d, 0x3a, 0xf6, 0x87,
0xbd, 0x61, 0x50, 0xa4, 0x24, 0xbd, 0xf9, 0x29, 0x82, 0x72, 0x9c, 0x04, 0x7f, 0x4f, 0xcb, 0x03,
0xc6, 0xee, 0xf4, 0xf3, 0xd9, 0x35, 0x79, 0x71, 0xf0, 0xde, 0x1b, 0xfb, 0xee, 0x9e, 0xca, 0x91,
0xda, 0x3b, 0x50, 0xd0, 0xd0, 0xac, 0x79, 0xec, 0x52, 0x15, 0xb3, 0xec, 0x33, 0x4c, 0xd6, 0xa4,
0x88, 0x63, 0x0e, 0x5c, 0x4a, 0x5e, 0x44, 0x2c, 0xe2, 0x4b, 0x11, 0x4f, 0xe2, 0x8b, 0x60, 0xf4,
0x5c, 0xc7, 0x5e, 0xc8, 0x4d, 0x7c, 0x07, 0xfe, 0x16, 0x24, 0x7d, 0x67, 0x21, 0x27, 0x25, 0x7d,
0x87, 0xf9, 0x48, 0x2a, 0x9f, 0x12, 0x37, 0x34, 0x01, 0x99, 0xbf, 0x45, 0xb0, 0xcc, 0xf6, 0x08,
0x0b, 0x6c, 0x0c, 0xa6, 0xe3, 0x5d, 0xbc, 0x06, 0x65, 0x76, 0xd2, 0x83, 0xa1, 0xec, 0x00, 0x0f,
0x86, 0x5d, 0xa9, 0xe6, 0x12, 0xc3, 0xab, 0xc6, 0xb0, 0xd5, 0xc5, 0xc7, 0x20, 0x3b, 0xf5, 0x04,
0x81, 0xd0, 0x39, 0xc3, 0xc0, 0xad, 0x2e, 0x3e, 0xa7, 0x1d, 0xc7, 0x6c, 0xad, 0x5d, 0x93, 0xb8,
0x0d, 0xef, 0x5a, 0x43, 0x37, 0x28, 0x3e, 0x67, 0x20, 0xd3, 0x61, 0x07, 0x8b, 0x38, 0x61, 0x1d,
0x28, 0x20, 0xe6, 0x02, 0x11, 0xb9, 0x6c, 0x7e, 0x1b, 0xf2, 0xc1, 0xee, 0xb9, 0x8d, 0x67, 0xae,
0x07, 0xcc, 0xcb, 0xb0, 0x2c, 0x8a, 0xea, 0xfc, 0xcd, 0xc5, 0x79, 0x9b, 0x8b, 0x6a, 0xf3, 0x71,
0x48, 0x0b, 0xab, 0x60, 0x30, 0xba, 0x96, 0x6f, 0xa9, 0x2d, 0xec, 0xdb, 0xac, 0xc2, 0xd1, 0x7b,
0xae, 0x35, 0xf6, 0x7a, 0xd4, 0xe5, 0x44, 0x41, 0xec, 0x9a, 0xaf, 0xc1, 0x0a, 0x2b, 0x24, 0xd4,
0xf5, 0x36, 0x9c, 0xe9, 0xd8, 0x57, 0x17, 0xfd, 0xf3, 0x50, 0x89, 0xa2, 0x65, 0xa8, 0x57, 0x20,
0xdd, 0x61, 0x08, 0xce, 0xbd, 0x44, 0x04, 0x60, 0xfe, 0x0a, 0x01, 0xbe, 0x46, 0x7d, 0xce, 0x7a,
0x6b, 0xd3, 0xd3, 0x2e, 0x77, 0xb6, 0xe5, 0x77, 0x06, 0xd4, 0xf5, 0xd4, 0x45, 0x47, 0xc1, 0xff,
0x8f, 0xcb, 0x9d, 0x79, 0x01, 0x56, 0x22, 0x52, 0x4a, 0x9d, 0x6a, 0x90, 0xeb, 0x48, 0x9c, 0x6c,
0xaa, 0x01, 0x6c, 0xfe, 0x3e, 0x09, 0x39, 0xe1, 0x5b, 0xda, 0xc3, 0x17, 0xa0, 0xd0, 0x63, 0xb1,
0xe6, 0x4e, 0xdc, 0xa1, 0x34, 0x81, 0xd1, 0x5e, 0x3e, 0x98, 0x35, 0x74, 0x34, 0xd1, 0x01, 0xfc,
0x46, 0x2c, 0xf0, 0xda, 0x95, 0xfd, 0x59, 0x23, 0xf3, 0x03, 0x16, 0x7c, 0x9b, 0xac, 0xbd, 0xf1,
0x30, 0xdc, 0x0c, 0xc2, 0xf1, 0x86, 0xcc, 0x36, 0x7e, 0xd3, 0x6b, 0x7f, 0x87, 0x89, 0x1f, 0xab,
0xd7, 0x13, 0xd7, 0xb1, 0xa9, 0x3f, 0xa0, 0x53, 0xaf, 0xd5, 0x71, 0x6c, 0xdb, 0x19, 0xb7, 0xf8,
0x58, 0xc7, 0x95, 0x66, 0x3d, 0x9a, 0x6d, 0x97, 0x09, 0x78, 0x0f, 0xb2, 0xfe, 0xc0, 0x75, 0xa6,
0xfd, 0x01, 0x6f, 0x3f, 0xa9, 0xf6, 0xa5, 0xc5, 0xf9, 0x29, 0x0e, 0x44, 0x7d, 0xe0, 0x53, 0xcc,
0x5a, 0xb4, 0xb3, 0xeb, 0x4d, 0x6d, 0x31, 0x2c, 0xb5, 0xd3, 0x07, 0xb3, 0x06, 0x7a, 0x83, 0x04,
0x68, 0xf3, 0xd3, 0x24, 0x34, 0x78, 0x08, 0xdf, 0xe7, 0x77, 0x93, 0xab, 0x8e, 0x7b, 0x8b, 0xfa,
0xee, 0xb0, 0x73, 0xdb, 0xb2, 0xa9, 0x8a, 0x8d, 0x06, 0x14, 0x6c, 0x8e, 0x7c, 0xa0, 0x25, 0x07,
0xd8, 0x01, 0x1d, 0x3e, 0x09, 0xc0, 0xd3, 0x4e, 0xac, 0x8b, 0x3c, 0xc9, 0x73, 0x0c, 0x5f, 0xde,
0x88, 0x58, 0xaa, 0xb5, 0xa0, 0x66, 0xd2, 0x42, 0x5b, 0x71, 0x0b, 0x2d, 0xcc, 0x27, 0x30, 0x8b,
0x1e, 0xeb, 0xe9, 0x68, 0xac, 0x9b, 0x7f, 0x43, 0x50, 0xbf, 0xa9, 0x24, 0x7f, 0x41, 0x73, 0x28,
0x7d, 0x93, 0x2f, 0x49, 0xdf, 0xd4, 0xff, 0xa6, 0xaf, 0xf9, 0x27, 0x2d, 0xe5, 0x09, 0xed, 0x29,
0x3d, 0x36, 0xb4, 0x76, 0xf1, 0x32, 0xc4, 0x4c, 0xbe, 0x44, 0xb7, 0xa4, 0x62, 0x6e, 0x79, 0x37,
0x2c, 0x07, 0x5c, 0x03, 0x59, 0x0e, 0x4e, 0x83, 0xe1, 0xd2, 0x9e, 0x6a, 0xbe, 0x38, 0x5e, 0xe3,
0x69, 0x8f, 0xf0, 0x75, 0xf3, 0x0f, 0x08, 0xca, 0xd7, 0xa8, 0x1f, 0xbd, 0xd6, 0xbc, 0x4a, 0xfa,
0xbf, 0x0f, 0x47, 0x34, 0xf9, 0xa5, 0xf6, 0x6f, 0xc5, 0xee, 0x32, 0xaf, 0x85, 0xfa, 0x6f, 0x8d,
0xbb, 0xf4, 0x13, 0x39, 0xa3, 0x45, 0xaf, 0x31, 0x77, 0xa1, 0xa0, 0x2d, 0xe2, 0x2b, 0xb1, 0x0b,
0xcc, 0xbc, 0xa6, 0xda, 0xae, 0x48, 0x9d, 0xc4, 0x94, 0x26, 0xaf, 0xa7, 0x41, 0xbb, 0xdf, 0x06,
0xcc, 0xc7, 0x46, 0xce, 0x56, 0xaf, 0xd4, 0x1c, 0x7b, 0x23, 0xb8, 0xcf, 0x04, 0x30, 0x3e, 0x05,
0x86, 0xeb, 0x3c, 0x52, 0x37, 0xd3, 0x52, 0x78, 0x24, 0x71, 0x1e, 0x11, 0xbe, 0x64, 0x5e, 0x86,
0x14, 0x71, 0x1e, 0xe1, 0x3a, 0x80, 0x6b, 0x8d, 0xfb, 0xf4, 0x7e, 0x30, 0xb0, 0x14, 0x89, 0x86,
0x79, 0x4e, 0x7f, 0xdd, 0x80, 0x23, 0xba, 0x44, 0xc2, 0xdd, 0x4d, 0xc8, 0x32, 0xe4, 0x70, 0xde,
0xa3, 0x17, 0x27, 0x14, 0xb3, 0xaf, 0x22, 0x62, 0x31, 0x03, 0x21, 0x1e, 0x9f, 0x80, 0xbc, 0x6f,
0xed, 0x8c, 0xe8, 0xed, 0x30, 0xe7, 0x43, 0x04, 0x5b, 0x65, 0xb3, 0xd6, 0x7d, 0xed, 0xa2, 0x10,
0x22, 0xf0, 0x59, 0x28, 0x87, 0x32, 0xdf, 0x75, 0x69, 0x6f, 0xf8, 0x09, 0xf7, 0x70, 0x91, 0x1c,
0xc2, 0xe3, 0x35, 0x58, 0x0e, 0x71, 0xdb, 0xbc, 0xed, 0x1a, 0x9c, 0x34, 0x8e, 0x66, 0xb6, 0xe1,
0xea, 0xbe, 0xf7, 0x70, 0x6a, 0x8d, 0x78, 0x21, 0x2b, 0x12, 0x0d, 0x63, 0xfe, 0x11, 0xc1, 0x11,
0xe1, 0x6a, 0x36, 0x65, 0xbf, 0x8a, 0x51, 0xff, 0x6b, 0x04, 0x58, 0xd7, 0x40, 0x86, 0xd6, 0x37,
0xf5, 0xe7, 0x13, 0xd6, 0xd7, 0x0b, 0x7c, 0x84, 0x14, 0xa8, 0xf0, 0x05, 0xc4, 0x0c, 0xae, 0x80,
0xfc, 0xdd, 0x51, 0xcc, 0xa8, 0x02, 0xa3, 0x6e, 0x7f, 0x6c, 0xb4, 0xde, 0xd9, 0xf3, 0xa9, 0x27,
0x27, 0x4c, 0x3e, 0x5a, 0x73, 0x04, 0x11, 0x3f, 0xec, 0x2c, 0x3a, 0xf6, 0x79, 0xd4, 0x18, 0xe1,
0x59, 0x12, 0x45, 0xd4, 0x07, 0x6b, 0x1b, 0xa5, 0xfb, 0xce, 0x68, 0x1a, 0x76, 0x89, 0x57, 0xc8,
0xce, 0xd1, 0xd1, 0x37, 0x2d, 0x47, 0x5f, 0xf3, 0x47, 0xb0, 0xa4, 0x54, 0x92, 0x86, 0x7f, 0x13,
0xb2, 0x1f, 0x73, 0xcc, 0x9c, 0x07, 0x21, 0x41, 0x2a, 0x8b, 0x8d, 0x22, 0x8b, 0xbe, 0xb3, 0x06,
0x9c, 0x77, 0x21, 0x23, 0xc8, 0xf1, 0x09, 0xfd, 0xc2, 0x2d, 0x5e, 0x2e, 0x18, 0x2c, 0x6f, 0xcf,
0x91, 0x87, 0x8f, 0xfc, 0x9c, 0x87, 0x0f, 0x13, 0x32, 0xe2, 0x24, 0xe9, 0x3f, 0xee, 0x62, 0x81,
0x21, 0xf2, 0xf7, 0xec, 0x69, 0xc8, 0x07, 0xaf, 0xa8, 0xb8, 0x00, 0xd9, 0xab, 0x77, 0xc8, 0x0f,
0xaf, 0x90, 0xcd, 0x72, 0x02, 0x17, 0x21, 0xd7, 0xbe, 0xb2, 0x71, 0x83, 0x43, 0x68, 0xfd, 0xdf,
0x86, 0x2a, 0x10, 0x2e, 0xfe, 0x2e, 0xa4, 0x45, 0xd6, 0x1f, 0x0d, 0x15, 0xd4, 0xdf, 0x42, 0x6b,
0xc7, 0x0e, 0xe1, 0xe5, 0x1d, 0x3d, 0xf1, 0x26, 0xc2, 0xb7, 0xa1, 0xc0, 0x91, 0xf2, 0xbd, 0xe5,
0x44, 0xfc, 0xd9, 0x23, 0xc2, 0xe9, 0xe4, 0x73, 0x56, 0x35, 0x7e, 0x97, 0x20, 0xcd, 0x0b, 0xaf,
0x2e, 0x8d, 0xfe, 0x5e, 0xa6, 0x4b, 0x13, 0x79, 0x81, 0x32, 0x13, 0xf8, 0x1d, 0x30, 0xd8, 0x70,
0x80, 0xb5, 0xde, 0xa0, 0x3d, 0x93, 0xd4, 0x8e, 0xc6, 0xd1, 0xda, 0xb1, 0xef, 0x06, 0xaf, 0x3d,
0xc7, 0xe2, 0x53, 0xad, 0xda, 0x5e, 0x3d, 0xbc, 0x10, 0x9c, 0x7c, 0x47, 0x3c, 0x7b, 0xa8, 0xb1,
0x04, 0x9f, 0x8c, 0x1e, 0x15, 0x9b, 0x62, 0x6a, 0xf5, 0xe7, 0x2d, 0x07, 0x0c, 0x6f, 0x42, 0x41,
0x1b, 0x09, 0x74, 0xb3, 0x1e, 0x9e, 0x67, 0x74, 0xb3, 0xce, 0x99, 0x23, 0xcc, 0x04, 0xbe, 0x06,
0x39, 0xd6, 0x51, 0x59, 0x61, 0xc1, 0xc7, 0xe3, 0x8d, 0x53, 0x2b, 0x98, 0xb5, 0x13, 0xf3, 0x17,
0x03, 0x46, 0x57, 0x61, 0x39, 0x68, 0xcd, 0x32, 0xaa, 0x8f, 0xc5, 0xd3, 0x62, 0x8e, 0xbd, 0xa2,
0xa9, 0x65, 0x26, 0xd6, 0x3f, 0x82, 0x9c, 0x9a, 0x82, 0xf1, 0x07, 0xb0, 0x14, 0x9d, 0x01, 0xf1,
0x37, 0x34, 0xf3, 0x44, 0x47, 0xeb, 0xda, 0xaa, 0xb6, 0x34, 0x7f, 0x70, 0x4c, 0xac, 0xa1, 0xf5,
0x8f, 0xd4, 0xbf, 0x6f, 0x36, 0x2d, 0xdf, 0xc2, 0x77, 0x60, 0x89, 0x6b, 0x1f, 0xfc, 0x7f, 0x27,
0x12, 0xa5, 0x87, 0xfe, 0x99, 0x14, 0x89, 0xd2, 0xc3, 0xff, 0x54, 0x32, 0x13, 0xed, 0x0f, 0x9f,
0x3c, 0xad, 0x27, 0x3e, 0x7f, 0x5a, 0x4f, 0x7c, 0xf1, 0xb4, 0x8e, 0x7e, 0xba, 0x5f, 0x47, 0xbf,
0xd9, 0xaf, 0xa3, 0xc7, 0xfb, 0x75, 0xf4, 0x64, 0xbf, 0x8e, 0xfe, 0xb9, 0x5f, 0x47, 0xff, 0xda,
0xaf, 0x27, 0xbe, 0xd8, 0xaf, 0xa3, 0xcf, 0x9e, 0xd5, 0x13, 0x4f, 0x9e, 0xd5, 0x13, 0x9f, 0x3f,
0xab, 0x27, 0x3e, 0x7c, 0xfd, 0xcb, 0x5e, 0xb7, 0xd4, 0x89, 0x3b, 0x19, 0xfe, 0xf3, 0xd6, 0x7f,
0x03, 0x00, 0x00, 0xff, 0xff, 0x0a, 0x2d, 0xf5, 0x83, 0x7d, 0x1b, 0x00, 0x00,
}
func (x Direction) String() string {
@ -3974,14 +3974,14 @@ func (this *IndexStatsResponse) Equal(that interface{}) bool {
}
return true
}
func (this *LabelVolumeRequest) Equal(that interface{}) bool {
func (this *VolumeRequest) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LabelVolumeRequest)
that1, ok := that.(*VolumeRequest)
if !ok {
that2, ok := that.(LabelVolumeRequest)
that2, ok := that.(VolumeRequest)
if ok {
that1 = &that2
} else {
@ -4007,14 +4007,14 @@ func (this *LabelVolumeRequest) Equal(that interface{}) bool {
}
return true
}
func (this *LabelVolumeResponse) Equal(that interface{}) bool {
func (this *VolumeResponse) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LabelVolumeResponse)
that1, ok := that.(*VolumeResponse)
if !ok {
that2, ok := that.(LabelVolumeResponse)
that2, ok := that.(VolumeResponse)
if ok {
that1 = &that2
} else {
@ -4039,14 +4039,14 @@ func (this *LabelVolumeResponse) Equal(that interface{}) bool {
}
return true
}
func (this *LabelVolume) Equal(that interface{}) bool {
func (this *Volume) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LabelVolume)
that1, ok := that.(*Volume)
if !ok {
that2, ok := that.(LabelVolume)
that2, ok := that.(Volume)
if ok {
that1 = &that2
} else {
@ -4602,12 +4602,12 @@ func (this *IndexStatsResponse) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LabelVolumeRequest) GoString() string {
func (this *VolumeRequest) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 8)
s = append(s, "&logproto.LabelVolumeRequest{")
s = append(s, "&logproto.VolumeRequest{")
s = append(s, "From: "+fmt.Sprintf("%#v", this.From)+",\n")
s = append(s, "Through: "+fmt.Sprintf("%#v", this.Through)+",\n")
s = append(s, "Matchers: "+fmt.Sprintf("%#v", this.Matchers)+",\n")
@ -4615,14 +4615,14 @@ func (this *LabelVolumeRequest) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LabelVolumeResponse) GoString() string {
func (this *VolumeResponse) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 6)
s = append(s, "&logproto.LabelVolumeResponse{")
s = append(s, "&logproto.VolumeResponse{")
if this.Volumes != nil {
vs := make([]*LabelVolume, len(this.Volumes))
vs := make([]*Volume, len(this.Volumes))
for i := range vs {
vs[i] = &this.Volumes[i]
}
@ -4632,12 +4632,12 @@ func (this *LabelVolumeResponse) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LabelVolume) GoString() string {
func (this *Volume) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 7)
s = append(s, "&logproto.LabelVolume{")
s = append(s, "&logproto.Volume{")
s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n")
s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n")
s = append(s, "Volume: "+fmt.Sprintf("%#v", this.Volume)+",\n")
@ -4677,7 +4677,7 @@ type QuerierClient interface {
GetStats(ctx context.Context, in *IndexStatsRequest, opts ...grpc.CallOption) (*IndexStatsResponse, error)
// Note: this MUST be the same as the variant defined in
// indexgateway.proto on the IndexGateway service.
GetLabelVolume(ctx context.Context, in *LabelVolumeRequest, opts ...grpc.CallOption) (*LabelVolumeResponse, error)
GetSeriesVolume(ctx context.Context, in *VolumeRequest, opts ...grpc.CallOption) (*VolumeResponse, error)
}
type querierClient struct {
@ -4829,9 +4829,9 @@ func (c *querierClient) GetStats(ctx context.Context, in *IndexStatsRequest, opt
return out, nil
}
func (c *querierClient) GetLabelVolume(ctx context.Context, in *LabelVolumeRequest, opts ...grpc.CallOption) (*LabelVolumeResponse, error) {
out := new(LabelVolumeResponse)
err := c.cc.Invoke(ctx, "/logproto.Querier/GetLabelVolume", in, out, opts...)
func (c *querierClient) GetSeriesVolume(ctx context.Context, in *VolumeRequest, opts ...grpc.CallOption) (*VolumeResponse, error) {
out := new(VolumeResponse)
err := c.cc.Invoke(ctx, "/logproto.Querier/GetSeriesVolume", in, out, opts...)
if err != nil {
return nil, err
}
@ -4852,7 +4852,7 @@ type QuerierServer interface {
GetStats(context.Context, *IndexStatsRequest) (*IndexStatsResponse, error)
// Note: this MUST be the same as the variant defined in
// indexgateway.proto on the IndexGateway service.
GetLabelVolume(context.Context, *LabelVolumeRequest) (*LabelVolumeResponse, error)
GetSeriesVolume(context.Context, *VolumeRequest) (*VolumeResponse, error)
}
// UnimplementedQuerierServer can be embedded to have forward compatible implementations.
@ -4883,8 +4883,8 @@ func (*UnimplementedQuerierServer) GetChunkIDs(ctx context.Context, req *GetChun
func (*UnimplementedQuerierServer) GetStats(ctx context.Context, req *IndexStatsRequest) (*IndexStatsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetStats not implemented")
}
func (*UnimplementedQuerierServer) GetLabelVolume(ctx context.Context, req *LabelVolumeRequest) (*LabelVolumeResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetLabelVolume not implemented")
func (*UnimplementedQuerierServer) GetSeriesVolume(ctx context.Context, req *VolumeRequest) (*VolumeResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetSeriesVolume not implemented")
}
func RegisterQuerierServer(s *grpc.Server, srv QuerierServer) {
@ -5044,20 +5044,20 @@ func _Querier_GetStats_Handler(srv interface{}, ctx context.Context, dec func(in
return interceptor(ctx, in, info, handler)
}
func _Querier_GetLabelVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LabelVolumeRequest)
func _Querier_GetSeriesVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(VolumeRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(QuerierServer).GetLabelVolume(ctx, in)
return srv.(QuerierServer).GetSeriesVolume(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/logproto.Querier/GetLabelVolume",
FullMethod: "/logproto.Querier/GetSeriesVolume",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(QuerierServer).GetLabelVolume(ctx, req.(*LabelVolumeRequest))
return srv.(QuerierServer).GetSeriesVolume(ctx, req.(*VolumeRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -5087,8 +5087,8 @@ var _Querier_serviceDesc = grpc.ServiceDesc{
Handler: _Querier_GetStats_Handler,
},
{
MethodName: "GetLabelVolume",
Handler: _Querier_GetLabelVolume_Handler,
MethodName: "GetSeriesVolume",
Handler: _Querier_GetSeriesVolume_Handler,
},
},
Streams: []grpc.StreamDesc{
@ -7096,7 +7096,7 @@ func (m *IndexStatsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LabelVolumeRequest) Marshal() (dAtA []byte, err error) {
func (m *VolumeRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
@ -7106,12 +7106,12 @@ func (m *LabelVolumeRequest) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
func (m *LabelVolumeRequest) MarshalTo(dAtA []byte) (int, error) {
func (m *VolumeRequest) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LabelVolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
func (m *VolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
@ -7141,7 +7141,7 @@ func (m *LabelVolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LabelVolumeResponse) Marshal() (dAtA []byte, err error) {
func (m *VolumeResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
@ -7151,12 +7151,12 @@ func (m *LabelVolumeResponse) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
func (m *LabelVolumeResponse) MarshalTo(dAtA []byte) (int, error) {
func (m *VolumeResponse) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LabelVolumeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
func (m *VolumeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
@ -7183,7 +7183,7 @@ func (m *LabelVolumeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LabelVolume) Marshal() (dAtA []byte, err error) {
func (m *Volume) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
@ -7193,12 +7193,12 @@ func (m *LabelVolume) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
func (m *LabelVolume) MarshalTo(dAtA []byte) (int, error) {
func (m *Volume) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LabelVolume) MarshalToSizedBuffer(dAtA []byte) (int, error) {
func (m *Volume) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
@ -8026,7 +8026,7 @@ func (m *IndexStatsResponse) Size() (n int) {
return n
}
func (m *LabelVolumeRequest) Size() (n int) {
func (m *VolumeRequest) Size() (n int) {
if m == nil {
return 0
}
@ -8048,7 +8048,7 @@ func (m *LabelVolumeRequest) Size() (n int) {
return n
}
func (m *LabelVolumeResponse) Size() (n int) {
func (m *VolumeResponse) Size() (n int) {
if m == nil {
return 0
}
@ -8066,7 +8066,7 @@ func (m *LabelVolumeResponse) Size() (n int) {
return n
}
func (m *LabelVolume) Size() (n int) {
func (m *Volume) Size() (n int) {
if m == nil {
return 0
}
@ -8647,11 +8647,11 @@ func (this *IndexStatsResponse) String() string {
}, "")
return s
}
func (this *LabelVolumeRequest) String() string {
func (this *VolumeRequest) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LabelVolumeRequest{`,
s := strings.Join([]string{`&VolumeRequest{`,
`From:` + fmt.Sprintf("%v", this.From) + `,`,
`Through:` + fmt.Sprintf("%v", this.Through) + `,`,
`Matchers:` + fmt.Sprintf("%v", this.Matchers) + `,`,
@ -8660,27 +8660,27 @@ func (this *LabelVolumeRequest) String() string {
}, "")
return s
}
func (this *LabelVolumeResponse) String() string {
func (this *VolumeResponse) String() string {
if this == nil {
return "nil"
}
repeatedStringForVolumes := "[]LabelVolume{"
repeatedStringForVolumes := "[]Volume{"
for _, f := range this.Volumes {
repeatedStringForVolumes += strings.Replace(strings.Replace(f.String(), "LabelVolume", "LabelVolume", 1), `&`, ``, 1) + ","
repeatedStringForVolumes += strings.Replace(strings.Replace(f.String(), "Volume", "Volume", 1), `&`, ``, 1) + ","
}
repeatedStringForVolumes += "}"
s := strings.Join([]string{`&LabelVolumeResponse{`,
s := strings.Join([]string{`&VolumeResponse{`,
`Volumes:` + repeatedStringForVolumes + `,`,
`Limit:` + fmt.Sprintf("%v", this.Limit) + `,`,
`}`,
}, "")
return s
}
func (this *LabelVolume) String() string {
func (this *Volume) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LabelVolume{`,
s := strings.Join([]string{`&Volume{`,
`Name:` + fmt.Sprintf("%v", this.Name) + `,`,
`Value:` + fmt.Sprintf("%v", this.Value) + `,`,
`Volume:` + fmt.Sprintf("%v", this.Volume) + `,`,
@ -14029,7 +14029,7 @@ func (m *IndexStatsResponse) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
func (m *VolumeRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
@ -14052,10 +14052,10 @@ func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LabelVolumeRequest: wiretype end group for non-group")
return fmt.Errorf("proto: VolumeRequest: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LabelVolumeRequest: illegal tag %d (wire type %d)", fieldNum, wire)
return fmt.Errorf("proto: VolumeRequest: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
@ -14171,7 +14171,7 @@ func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
func (m *VolumeResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
@ -14194,10 +14194,10 @@ func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LabelVolumeResponse: wiretype end group for non-group")
return fmt.Errorf("proto: VolumeResponse: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LabelVolumeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
return fmt.Errorf("proto: VolumeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
@ -14229,7 +14229,7 @@ func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Volumes = append(m.Volumes, LabelVolume{})
m.Volumes = append(m.Volumes, Volume{})
if err := m.Volumes[len(m.Volumes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
@ -14277,7 +14277,7 @@ func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LabelVolume) Unmarshal(dAtA []byte) error {
func (m *Volume) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
@ -14300,10 +14300,10 @@ func (m *LabelVolume) Unmarshal(dAtA []byte) error {
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LabelVolume: wiretype end group for non-group")
return fmt.Errorf("proto: Volume: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LabelVolume: illegal tag %d (wire type %d)", fieldNum, wire)
return fmt.Errorf("proto: Volume: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:

@ -29,7 +29,7 @@ service Querier {
rpc GetStats(IndexStatsRequest) returns (IndexStatsResponse) {}
// Note: this MUST be the same as the variant defined in
// indexgateway.proto on the IndexGateway service.
rpc GetLabelVolume(LabelVolumeRequest) returns (LabelVolumeResponse) {}
rpc GetSeriesVolume(VolumeRequest) returns (VolumeResponse) {}
}
service Ingester {
@ -381,7 +381,7 @@ message IndexStatsResponse {
uint64 entries = 4 [(gogoproto.jsontag) = "entries"];
}
message LabelVolumeRequest {
message VolumeRequest {
int64 from = 1 [
(gogoproto.customtype) = "github.com/prometheus/common/model.Time",
(gogoproto.nullable) = false
@ -394,12 +394,12 @@ message LabelVolumeRequest {
int32 limit = 4;
}
message LabelVolumeResponse {
repeated LabelVolume volumes = 1 [(gogoproto.nullable) = false];
message VolumeResponse {
repeated Volume volumes = 1 [(gogoproto.nullable) = false];
int32 limit = 2;
}
message LabelVolume {
message Volume {
string name = 1 [(gogoproto.jsontag) = "name"];
string value = 2 [(gogoproto.jsontag) = "value"];
uint64 volume = 3 [(gogoproto.jsontag) = "volume"];

@ -416,9 +416,9 @@ func (t *Loki) initQuerier() (services.Service, error) {
"/loki/api/v1/labels": labelsHTTPMiddleware.Wrap(http.HandlerFunc(t.querierAPI.LabelHandler)),
"/loki/api/v1/label/{name}/values": labelsHTTPMiddleware.Wrap(http.HandlerFunc(t.querierAPI.LabelHandler)),
"/loki/api/v1/series": querier.WrapQuerySpanAndTimeout("query.Series", t.querierAPI).Wrap(http.HandlerFunc(t.querierAPI.SeriesHandler)),
"/loki/api/v1/index/stats": indexStatsHTTPMiddleware.Wrap(http.HandlerFunc(t.querierAPI.IndexStatsHandler)),
"/loki/api/v1/index/label_volume": querier.WrapQuerySpanAndTimeout("query.IndexStats", t.querierAPI).Wrap(http.HandlerFunc(t.querierAPI.LabelVolumeHandler)),
"/loki/api/v1/series": querier.WrapQuerySpanAndTimeout("query.Series", t.querierAPI).Wrap(http.HandlerFunc(t.querierAPI.SeriesHandler)),
"/loki/api/v1/index/stats": indexStatsHTTPMiddleware.Wrap(http.HandlerFunc(t.querierAPI.IndexStatsHandler)),
"/loki/api/v1/index/series_volume": querier.WrapQuerySpanAndTimeout("query.SeriesVolume", t.querierAPI).Wrap(http.HandlerFunc(t.querierAPI.SeriesVolumeHandler)),
"/api/prom/query": middleware.Merge(
httpMiddleware,
@ -868,7 +868,7 @@ func (t *Loki) initQueryFrontend() (_ services.Service, err error) {
t.Server.HTTP.Path("/loki/api/v1/label/{name}/values").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/loki/api/v1/series").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/loki/api/v1/index/stats").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/loki/api/v1/index/label_volume").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/loki/api/v1/index/series_volume").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/api/prom/query").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/api/prom/label").Methods("GET", "POST").Handler(frontendHandler)
t.Server.HTTP.Path("/api/prom/label/{name}/values").Methods("GET", "POST").Handler(frontendHandler)

@ -442,32 +442,32 @@ func (q *QuerierAPI) IndexStatsHandler(w http.ResponseWriter, r *http.Request) {
}
}
// LabelVolumeHandler queries the index label volumes related to the passed matchers
func (q *QuerierAPI) LabelVolumeHandler(w http.ResponseWriter, r *http.Request) {
rawReq, err := loghttp.ParseLabelVolumeQuery(r)
// SeriesVolumeHandler queries the index label volumes related to the passed matchers
func (q *QuerierAPI) SeriesVolumeHandler(w http.ResponseWriter, r *http.Request) {
rawReq, err := loghttp.ParseSeriesVolumeQuery(r)
if err != nil {
serverutil.WriteError(httpgrpc.Errorf(http.StatusBadRequest, err.Error()), w)
return
}
req := &logproto.LabelVolumeRequest{
req := &logproto.VolumeRequest{
From: model.TimeFromUnixNano(rawReq.Start.UnixNano()),
Through: model.TimeFromUnixNano(rawReq.End.UnixNano()),
Matchers: rawReq.Query,
Limit: int32(rawReq.Limit),
}
resp, err := q.querier.LabelVolume(r.Context(), req)
resp, err := q.querier.SeriesVolume(r.Context(), req)
if err != nil {
serverutil.WriteError(err, w)
return
}
if resp == nil { // Some stores don't implement this
resp = &logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{}}
resp = &logproto.VolumeResponse{Volumes: []logproto.Volume{}}
}
if marshal.WriteLabelVolumeResponseJSON(resp, w) != nil {
if marshal.WriteSeriesVolumeResponseJSON(resp, w) != nil {
serverutil.WriteError(err, w)
return
}

@ -198,27 +198,27 @@ func TestQueryWrapperMiddleware(t *testing.T) {
})
}
func TestLabelVolumeHandler(t *testing.T) {
func TestSeriesVolumeHandler(t *testing.T) {
t.Run("it returns label volumes from the querier", func(t *testing.T) {
ret := &logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{
ret := &logproto.VolumeResponse{Volumes: []logproto.Volume{
{Name: "foo", Value: "bar", Volume: 38},
}}
querier := newQuerierMock()
querier.On("LabelVolume", mock.Anything, mock.Anything).Return(ret, nil)
querier.On("SeriesVolume", mock.Anything, mock.Anything).Return(ret, nil)
api := NewQuerierAPI(Config{}, querier, nil, log.NewNopLogger())
req := httptest.NewRequest(http.MethodGet, "/label_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
req := httptest.NewRequest(http.MethodGet, "/series_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
err := req.ParseForm()
require.NoError(t, err)
w := httptest.NewRecorder()
api.LabelVolumeHandler(w, req)
api.SeriesVolumeHandler(w, req)
calls := querier.GetMockedCallsByMethod("LabelVolume")
calls := querier.GetMockedCallsByMethod("SeriesVolume")
require.Len(t, calls, 1)
require.Equal(t, &logproto.LabelVolumeRequest{
require.Equal(t, &logproto.VolumeRequest{
From: 0,
Through: 1000,
Matchers: `{foo="bar"}`,
@ -231,18 +231,18 @@ func TestLabelVolumeHandler(t *testing.T) {
t.Run("it returns nothing when a store doesn't support label volumes", func(t *testing.T) {
querier := newQuerierMock()
querier.On("LabelVolume", mock.Anything, mock.Anything).Return(nil, nil)
querier.On("SeriesVolume", mock.Anything, mock.Anything).Return(nil, nil)
api := NewQuerierAPI(Config{}, querier, nil, log.NewNopLogger())
req := httptest.NewRequest(http.MethodGet, "/label_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
req := httptest.NewRequest(http.MethodGet, "/series_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
err := req.ParseForm()
require.NoError(t, err)
w := httptest.NewRecorder()
api.LabelVolumeHandler(w, req)
api.SeriesVolumeHandler(w, req)
calls := querier.GetMockedCallsByMethod("LabelVolume")
calls := querier.GetMockedCallsByMethod("SeriesVolume")
require.Len(t, calls, 1)
require.Equal(t, strings.TrimSpace(w.Body.String()), `{"volumes":[]}`)
@ -251,18 +251,18 @@ func TestLabelVolumeHandler(t *testing.T) {
t.Run("it returns error when there's an error in the querier", func(t *testing.T) {
querier := newQuerierMock()
querier.On("LabelVolume", mock.Anything, mock.Anything).Return(nil, errors.New("something bad"))
querier.On("SeriesVolume", mock.Anything, mock.Anything).Return(nil, errors.New("something bad"))
api := NewQuerierAPI(Config{}, querier, nil, log.NewNopLogger())
req := httptest.NewRequest(http.MethodGet, "/label_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
req := httptest.NewRequest(http.MethodGet, "/series_volume?start=0&end=1&query=%7Bfoo%3D%22bar%22%7D", nil)
err := req.ParseForm()
require.NoError(t, err)
w := httptest.NewRecorder()
api.LabelVolumeHandler(w, req)
api.SeriesVolumeHandler(w, req)
calls := querier.GetMockedCallsByMethod("LabelVolume")
calls := querier.GetMockedCallsByMethod("SeriesVolume")
require.Len(t, calls, 1)
require.Equal(t, strings.TrimSpace(w.Body.String()), `something bad`)

@ -6,7 +6,7 @@ import (
"strings"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/gogo/status"
"github.com/grafana/dskit/ring"
@ -319,14 +319,14 @@ func (q *IngesterQuerier) Stats(ctx context.Context, _ string, from, through mod
return &merged, nil
}
func (q *IngesterQuerier) LabelVolume(ctx context.Context, _ string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (q *IngesterQuerier) SeriesVolume(ctx context.Context, _ string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
matcherString := "{}"
if len(matchers) > 0 {
matcherString = syntax.MatchersString(matchers)
}
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) {
return querierClient.GetLabelVolume(ctx, &logproto.LabelVolumeRequest{
return querierClient.GetSeriesVolume(ctx, &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: matcherString,
@ -337,17 +337,17 @@ func (q *IngesterQuerier) LabelVolume(ctx context.Context, _ string, from, throu
if err != nil {
if isUnimplementedCallError(err) {
// Handle communication with older ingesters gracefully
return &logproto.LabelVolumeResponse{}, nil
return &logproto.VolumeResponse{}, nil
}
return nil, err
}
casted := make([]*logproto.LabelVolumeResponse, 0, len(resps))
casted := make([]*logproto.VolumeResponse, 0, len(resps))
for _, resp := range resps {
casted = append(casted, resp.response.(*logproto.LabelVolumeResponse))
casted = append(casted, resp.response.(*logproto.VolumeResponse))
}
merged := labelvolume.Merge(casted, limit)
merged := seriesvolume.Merge(casted, limit)
return merged, nil
}

@ -345,17 +345,17 @@ func TestConvertMatchersToString(t *testing.T) {
}
}
func TestIngesterQuerier_LabelVolume(t *testing.T) {
func TestIngesterQuerier_SeriesVolume(t *testing.T) {
t.Run("it gets label volumes from all the ingesters", func(t *testing.T) {
ret := &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
ret := &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
Limit: 10,
}
ingesterClient := newQuerierClientMock()
ingesterClient.On("GetLabelVolume", mock.Anything, mock.Anything, mock.Anything).Return(ret, nil)
ingesterClient.On("GetSeriesVolume", mock.Anything, mock.Anything, mock.Anything).Return(ret, nil)
ingesterQuerier, err := newIngesterQuerier(
mockIngesterClientConfig(),
@ -365,17 +365,17 @@ func TestIngesterQuerier_LabelVolume(t *testing.T) {
)
require.NoError(t, err)
volumes, err := ingesterQuerier.LabelVolume(context.Background(), "", 0, 1, 10)
volumes, err := ingesterQuerier.SeriesVolume(context.Background(), "", 0, 1, 10)
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 76},
require.Equal(t, []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 76},
}, volumes.Volumes)
})
t.Run("it returns an empty result when an unimplemented error happens", func(t *testing.T) {
ingesterClient := newQuerierClientMock()
ingesterClient.On("GetLabelVolume", mock.Anything, mock.Anything, mock.Anything).Return(nil, status.Error(codes.Unimplemented, "something bad"))
ingesterClient.On("GetSeriesVolume", mock.Anything, mock.Anything, mock.Anything).Return(nil, status.Error(codes.Unimplemented, "something bad"))
ingesterQuerier, err := newIngesterQuerier(
mockIngesterClientConfig(),
@ -385,9 +385,9 @@ func TestIngesterQuerier_LabelVolume(t *testing.T) {
)
require.NoError(t, err)
volumes, err := ingesterQuerier.LabelVolume(context.Background(), "", 0, 1, 10)
volumes, err := ingesterQuerier.SeriesVolume(context.Background(), "", 0, 1, 10)
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume(nil), volumes.Volumes)
require.Equal(t, []logproto.Volume(nil), volumes.Volumes)
})
}

@ -3,7 +3,7 @@ package querier
import (
"context"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log"
"github.com/prometheus/prometheus/model/labels"
@ -188,16 +188,16 @@ func (q *MultiTenantQuerier) IndexStats(ctx context.Context, req *loghttp.RangeQ
return &merged, nil
}
func (q *MultiTenantQuerier) LabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
func (q *MultiTenantQuerier) SeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
tenantIDs, err := tenant.TenantIDs(ctx)
if err != nil {
return nil, err
}
responses := make([]*logproto.LabelVolumeResponse, len(tenantIDs))
responses := make([]*logproto.VolumeResponse, len(tenantIDs))
for i, id := range tenantIDs {
singleContext := user.InjectOrgID(ctx, id)
resp, err := q.Querier.LabelVolume(singleContext, req)
resp, err := q.Querier.SeriesVolume(singleContext, req)
if err != nil {
return nil, err
}
@ -205,7 +205,7 @@ func (q *MultiTenantQuerier) LabelVolume(ctx context.Context, req *logproto.Labe
responses[i] = resp
}
merged := labelvolume.Merge(responses, req.Limit)
merged := seriesvolume.Merge(responses, req.Limit)
return merged, nil
}

@ -407,39 +407,39 @@ func TestMultiTenantQuerierSeries(t *testing.T) {
}
}
func TestLabelVolume(t *testing.T) {
func TestSeriesVolume(t *testing.T) {
tenant.WithDefaultResolver(tenant.NewMultiResolver())
for _, tc := range []struct {
desc string
orgID string
expectedLabelVolumes []logproto.LabelVolume
desc string
orgID string
expectedSeriesVolumes []logproto.Volume
}{
{
desc: "multiple tenants are aggregated",
orgID: "1|2",
expectedLabelVolumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 76},
expectedSeriesVolumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 76},
},
},
{
desc: "single tenant",
orgID: "2",
expectedLabelVolumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
expectedSeriesVolumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
},
} {
t.Run(tc.desc, func(t *testing.T) {
querier := newQuerierMock()
querier.On("LabelVolume", mock.Anything, mock.Anything).Return(mockLabelValueResponse(), nil)
querier.On("SeriesVolume", mock.Anything, mock.Anything).Return(mockLabelValueResponse(), nil)
multiTenantQuerier := NewMultiTenantQuerier(querier, log.NewNopLogger())
ctx := user.InjectOrgID(context.Background(), tc.orgID)
resp, err := multiTenantQuerier.LabelVolume(ctx, mockLabelValueRequest())
resp, err := multiTenantQuerier.SeriesVolume(ctx, mockLabelValueRequest())
require.NoError(t, err)
require.Equal(t, tc.expectedLabelVolumes, resp.GetVolumes())
require.Equal(t, tc.expectedSeriesVolumes, resp.GetVolumes())
})
}
}
@ -470,8 +470,8 @@ func mockSeriesResponse() *logproto.SeriesResponse {
}
}
func mockLabelValueRequest() *logproto.LabelVolumeRequest {
return &logproto.LabelVolumeRequest{
func mockLabelValueRequest() *logproto.VolumeRequest {
return &logproto.VolumeRequest{
From: 0,
Through: 1000,
Matchers: `{foo="bar"}`,
@ -479,9 +479,9 @@ func mockLabelValueRequest() *logproto.LabelVolumeRequest {
}
}
func mockLabelValueResponse() *logproto.LabelVolumeResponse {
return &logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
func mockLabelValueResponse() *logproto.VolumeResponse {
return &logproto.VolumeResponse{Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
Limit: 10,
}

@ -6,7 +6,9 @@ import (
"net/http"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/opentracing/opentracing-go"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/tenant"
@ -88,7 +90,7 @@ type Querier interface {
Series(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error)
Tail(ctx context.Context, req *logproto.TailRequest) (*Tailer, error)
IndexStats(ctx context.Context, req *loghttp.RangeQuery) (*stats.Stats, error)
LabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error)
SeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error)
}
type Limits interface {
@ -767,14 +769,17 @@ func (q *SingleTenantQuerier) IndexStats(ctx context.Context, req *loghttp.Range
)
}
func (q *SingleTenantQuerier) LabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
func (q *SingleTenantQuerier) SeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "Querier.SeriesVolume")
defer sp.Finish()
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
matchers, err := syntax.ParseMatchers(req.Matchers)
if err != nil && req.Matchers != labelvolume.MatchAny {
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
@ -783,13 +788,21 @@ func (q *SingleTenantQuerier) LabelVolume(ctx context.Context, req *logproto.Lab
// TODO: remove this clause once we remove the deprecated query-timeout flag.
if q.cfg.QueryTimeout != 0 { // querier YAML configuration.
level.Warn(util_log.Logger).Log("msg", "deprecated querier:query_timeout YAML configuration identified. Please migrate to limits:query_timeout instead.", "call", "SingleTenantQuerier/LabelVolume")
level.Warn(util_log.Logger).Log("msg", "deprecated querier:query_timeout YAML configuration identified. Please migrate to limits:query_timeout instead.", "call", "SingleTenantQuerier/Volume")
queryTimeout = q.cfg.QueryTimeout
}
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
return q.store.LabelVolume(
sp.LogKV(
"user", userID,
"from", req.From.Time(),
"through", req.Through.Time(),
"matchers", syntax.MatchersString(matchers),
"limit", req.Limit,
)
return q.store.SeriesVolume(
ctx,
userID,
req.From,

@ -105,13 +105,13 @@ func (c *querierClientMock) GetChunkIDs(ctx context.Context, in *logproto.GetChu
return res.(*logproto.GetChunkIDsResponse), args.Error(1)
}
func (c *querierClientMock) GetLabelVolume(ctx context.Context, in *logproto.LabelVolumeRequest, opts ...grpc.CallOption) (*logproto.LabelVolumeResponse, error) {
func (c *querierClientMock) GetSeriesVolume(ctx context.Context, in *logproto.VolumeRequest, opts ...grpc.CallOption) (*logproto.VolumeResponse, error) {
args := c.Called(ctx, in, opts)
res := args.Get(0)
if res == nil {
return (*logproto.LabelVolumeResponse)(nil), args.Error(1)
return (*logproto.VolumeResponse)(nil), args.Error(1)
}
return res.(*logproto.LabelVolumeResponse), args.Error(1)
return res.(*logproto.VolumeResponse), args.Error(1)
}
func (c *querierClientMock) Context() context.Context {
@ -365,9 +365,9 @@ func (s *storeMock) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*la
return nil, nil
}
func (s *storeMock) LabelVolume(ctx context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (s *storeMock) SeriesVolume(ctx context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
args := s.Called(ctx, userID, from, through, matchers)
return args.Get(0).(*logproto.LabelVolumeResponse), args.Error(1)
return args.Get(0).(*logproto.VolumeResponse), args.Error(1)
}
func (s *storeMock) Stop() {
@ -536,8 +536,8 @@ func (q *querierMock) IndexStats(_ context.Context, _ *loghttp.RangeQuery) (*sta
return nil, nil
}
func (q *querierMock) LabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
args := q.MethodCalled("LabelVolume", ctx, req)
func (q *querierMock) SeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
args := q.MethodCalled("SeriesVolume", ctx, req)
resp := args.Get(0)
err := args.Error(1)
@ -545,5 +545,5 @@ func (q *querierMock) LabelVolume(ctx context.Context, req *logproto.LabelVolume
return nil, err
}
return resp.(*logproto.LabelVolumeResponse), err
return resp.(*logproto.VolumeResponse), err
}

@ -968,7 +968,7 @@ func TestQuerier_RequestingIngesters(t *testing.T) {
func TestQuerier_LabeleVolumes(t *testing.T) {
t.Run("it returns label volumes from the store", func(t *testing.T) {
ret := &logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{
ret := &logproto.VolumeResponse{Volumes: []logproto.Volume{
{Name: "foo", Value: "bar", Volume: 38},
}}
@ -976,17 +976,17 @@ func TestQuerier_LabeleVolumes(t *testing.T) {
require.NoError(t, err)
store := newStoreMock()
store.On("LabelVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(ret, nil)
store.On("SeriesVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(ret, nil)
querier := SingleTenantQuerier{
store: store,
limits: limits,
}
req := &logproto.LabelVolumeRequest{From: 0, Through: 1000, Matchers: `{}`}
req := &logproto.VolumeRequest{From: 0, Through: 1000, Matchers: `{}`}
ctx := user.InjectOrgID(context.Background(), "test")
resp, err := querier.LabelVolume(ctx, req)
resp, err := querier.SeriesVolume(ctx, req)
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{{Name: "foo", Value: "bar", Volume: 38}}, resp.Volumes)
require.Equal(t, []logproto.Volume{{Name: "foo", Value: "bar", Volume: 38}}, resp.Volumes)
})
}

@ -13,7 +13,7 @@ import (
strings "strings"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
json "github.com/json-iterator/go"
"github.com/opentracing/opentracing-go"
@ -271,13 +271,13 @@ func (Codec) DecodeRequest(_ context.Context, r *http.Request, _ []string) (quer
Through: through,
Matchers: req.Query,
}, err
case LabelVolumeOp:
req, err := loghttp.ParseLabelVolumeQuery(r)
case SeriesVolumeOp:
req, err := loghttp.ParseSeriesVolumeQuery(r)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
from, through := util.RoundToMilliseconds(req.Start, req.End)
return &logproto.LabelVolumeRequest{
return &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: req.Query,
@ -414,7 +414,7 @@ func (Codec) EncodeRequest(ctx context.Context, r queryrangebase.Request) (*http
Header: header,
}
return req.WithContext(ctx), nil
case *logproto.LabelVolumeRequest:
case *logproto.VolumeRequest:
params := url.Values{
"start": []string{fmt.Sprintf("%d", request.From.Time().UnixNano())},
"end": []string{fmt.Sprintf("%d", request.Through.Time().UnixNano())},
@ -422,7 +422,7 @@ func (Codec) EncodeRequest(ctx context.Context, r queryrangebase.Request) (*http
"limit": []string{fmt.Sprintf("%d", request.Limit)},
}
u := &url.URL{
Path: "/loki/api/v1/index/label_volume",
Path: "/loki/api/v1/index/series_volume",
RawQuery: params.Encode(),
}
req := &http.Request{
@ -500,12 +500,12 @@ func (Codec) DecodeResponse(_ context.Context, r *http.Response, req queryrangeb
Response: &resp,
Headers: httpResponseHeadersToPromResponseHeaders(r.Header),
}, nil
case *logproto.LabelVolumeRequest:
var resp logproto.LabelVolumeResponse
case *logproto.VolumeRequest:
var resp logproto.VolumeResponse
if err := json.Unmarshal(buf, &resp); err != nil {
return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err)
}
return &LabelVolumeResponse{
return &VolumeResponse{
Response: &resp,
Headers: httpResponseHeadersToPromResponseHeaders(r.Header),
}, nil
@ -637,8 +637,8 @@ func (Codec) EncodeResponse(ctx context.Context, res queryrangebase.Response) (*
if err := marshal.WriteIndexStatsResponseJSON(response.Response, &buf); err != nil {
return nil, err
}
case *LabelVolumeResponse:
if err := marshal.WriteLabelVolumeResponseJSON(response.Response, &buf); err != nil {
case *VolumeResponse:
if err := marshal.WriteSeriesVolumeResponseJSON(response.Response, &buf); err != nil {
return nil, err
}
default:
@ -742,17 +742,17 @@ func (Codec) MergeResponse(responses ...queryrangebase.Response) (queryrangebase
Response: &mergedIndexStats,
Headers: headers,
}, nil
case *LabelVolumeResponse:
resp0 := responses[0].(*LabelVolumeResponse)
case *VolumeResponse:
resp0 := responses[0].(*VolumeResponse)
headers := resp0.Headers
resps := make([]*logproto.LabelVolumeResponse, 0, len(responses))
resps := make([]*logproto.VolumeResponse, 0, len(responses))
for _, r := range responses {
resps = append(resps, r.(*LabelVolumeResponse).Response)
resps = append(resps, r.(*VolumeResponse).Response)
}
return &LabelVolumeResponse{
Response: labelvolume.Merge(resps, resp0.Response.Limit),
return &VolumeResponse{
Response: seriesvolume.Merge(resps, resp0.Response.Limit),
Headers: headers,
}, nil

@ -101,26 +101,26 @@ func Test_codec_DecodeRequest(t *testing.T) {
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: `{job="foo"}`,
}, false},
{"label_volume", func() (*http.Request, error) {
return LokiCodec.EncodeRequest(context.Background(), &logproto.LabelVolumeRequest{
{"series_volume", func() (*http.Request, error) {
return LokiCodec.EncodeRequest(context.Background(), &logproto.VolumeRequest{
From: model.TimeFromUnixNano(start.UnixNano()),
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: `{job="foo"}`,
Limit: 3,
})
}, &logproto.LabelVolumeRequest{
}, &logproto.VolumeRequest{
From: model.TimeFromUnixNano(start.UnixNano()),
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: `{job="foo"}`,
Limit: 3,
}, false},
{"label_volume_default_limit", func() (*http.Request, error) {
return LokiCodec.EncodeRequest(context.Background(), &logproto.LabelVolumeRequest{
{"series_volume_default_limit", func() (*http.Request, error) {
return LokiCodec.EncodeRequest(context.Background(), &logproto.VolumeRequest{
From: model.TimeFromUnixNano(start.UnixNano()),
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: `{job="foo"}`,
})
}, &logproto.LabelVolumeRequest{
}, &logproto.VolumeRequest{
From: model.TimeFromUnixNano(start.UnixNano()),
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: `{job="foo"}`,
@ -261,10 +261,10 @@ func Test_codec_DecodeResponse(t *testing.T) {
},
{
"label volume", &http.Response{StatusCode: 200, Body: io.NopCloser(strings.NewReader(labelVolumeString))},
&logproto.LabelVolumeRequest{},
&LabelVolumeResponse{
Response: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
&logproto.VolumeRequest{},
&VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: "foo", Value: "bar", Volume: 38},
},
Limit: 100,
@ -519,9 +519,9 @@ func Test_codec_EncodeResponse(t *testing.T) {
},
{
"label volume",
&LabelVolumeResponse{
Response: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
&VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: "foo", Value: "bar", Volume: 38},
},
Limit: 100,

@ -63,7 +63,7 @@ func (m *IndexStatsResponse) GetHeaders() []*queryrangebase.PrometheusResponseHe
}
// GetHeaders returns the HTTP headers in the response.
func (m *LabelVolumeResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader {
func (m *VolumeResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader {
if m != nil {
return convertPrometheusResponseHeadersToPointers(m.Headers)
}

@ -738,23 +738,23 @@ func (m *IndexStatsResponse) XXX_DiscardUnknown() {
var xxx_messageInfo_IndexStatsResponse proto.InternalMessageInfo
type LabelVolumeRequest struct {
type VolumeRequest struct {
Match []string `protobuf:"bytes,1,rep,name=match,proto3" json:"match,omitempty"`
StartTs time.Time `protobuf:"bytes,2,opt,name=startTs,proto3,stdtime" json:"startTs"`
EndTs time.Time `protobuf:"bytes,3,opt,name=endTs,proto3,stdtime" json:"endTs"`
}
func (m *LabelVolumeRequest) Reset() { *m = LabelVolumeRequest{} }
func (*LabelVolumeRequest) ProtoMessage() {}
func (*LabelVolumeRequest) Descriptor() ([]byte, []int) {
func (m *VolumeRequest) Reset() { *m = VolumeRequest{} }
func (*VolumeRequest) ProtoMessage() {}
func (*VolumeRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{10}
}
func (m *LabelVolumeRequest) XXX_Unmarshal(b []byte) error {
func (m *VolumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LabelVolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
func (m *VolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LabelVolumeRequest.Marshal(b, m, deterministic)
return xxx_messageInfo_VolumeRequest.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
@ -764,55 +764,55 @@ func (m *LabelVolumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte,
return b[:n], nil
}
}
func (m *LabelVolumeRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_LabelVolumeRequest.Merge(m, src)
func (m *VolumeRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_VolumeRequest.Merge(m, src)
}
func (m *LabelVolumeRequest) XXX_Size() int {
func (m *VolumeRequest) XXX_Size() int {
return m.Size()
}
func (m *LabelVolumeRequest) XXX_DiscardUnknown() {
xxx_messageInfo_LabelVolumeRequest.DiscardUnknown(m)
func (m *VolumeRequest) XXX_DiscardUnknown() {
xxx_messageInfo_VolumeRequest.DiscardUnknown(m)
}
var xxx_messageInfo_LabelVolumeRequest proto.InternalMessageInfo
var xxx_messageInfo_VolumeRequest proto.InternalMessageInfo
func (m *LabelVolumeRequest) GetMatch() []string {
func (m *VolumeRequest) GetMatch() []string {
if m != nil {
return m.Match
}
return nil
}
func (m *LabelVolumeRequest) GetStartTs() time.Time {
func (m *VolumeRequest) GetStartTs() time.Time {
if m != nil {
return m.StartTs
}
return time.Time{}
}
func (m *LabelVolumeRequest) GetEndTs() time.Time {
func (m *VolumeRequest) GetEndTs() time.Time {
if m != nil {
return m.EndTs
}
return time.Time{}
}
type LabelVolumeResponse struct {
Response *github_com_grafana_loki_pkg_logproto.LabelVolumeResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/pkg/logproto.LabelVolumeResponse" json:"response,omitempty"`
type VolumeResponse struct {
Response *github_com_grafana_loki_pkg_logproto.VolumeResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/pkg/logproto.VolumeResponse" json:"response,omitempty"`
Headers []github_com_grafana_loki_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"`
}
func (m *LabelVolumeResponse) Reset() { *m = LabelVolumeResponse{} }
func (*LabelVolumeResponse) ProtoMessage() {}
func (*LabelVolumeResponse) Descriptor() ([]byte, []int) {
func (m *VolumeResponse) Reset() { *m = VolumeResponse{} }
func (*VolumeResponse) ProtoMessage() {}
func (*VolumeResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_51b9d53b40d11902, []int{11}
}
func (m *LabelVolumeResponse) XXX_Unmarshal(b []byte) error {
func (m *VolumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *LabelVolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
func (m *VolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_LabelVolumeResponse.Marshal(b, m, deterministic)
return xxx_messageInfo_VolumeResponse.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
@ -822,17 +822,17 @@ func (m *LabelVolumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte,
return b[:n], nil
}
}
func (m *LabelVolumeResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_LabelVolumeResponse.Merge(m, src)
func (m *VolumeResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_VolumeResponse.Merge(m, src)
}
func (m *LabelVolumeResponse) XXX_Size() int {
func (m *VolumeResponse) XXX_Size() int {
return m.Size()
}
func (m *LabelVolumeResponse) XXX_DiscardUnknown() {
xxx_messageInfo_LabelVolumeResponse.DiscardUnknown(m)
func (m *VolumeResponse) XXX_DiscardUnknown() {
xxx_messageInfo_VolumeResponse.DiscardUnknown(m)
}
var xxx_messageInfo_LabelVolumeResponse proto.InternalMessageInfo
var xxx_messageInfo_VolumeResponse proto.InternalMessageInfo
func init() {
proto.RegisterType((*LokiRequest)(nil), "queryrange.LokiRequest")
@ -845,8 +845,8 @@ func init() {
proto.RegisterType((*LokiData)(nil), "queryrange.LokiData")
proto.RegisterType((*LokiPromResponse)(nil), "queryrange.LokiPromResponse")
proto.RegisterType((*IndexStatsResponse)(nil), "queryrange.IndexStatsResponse")
proto.RegisterType((*LabelVolumeRequest)(nil), "queryrange.LabelVolumeRequest")
proto.RegisterType((*LabelVolumeResponse)(nil), "queryrange.LabelVolumeResponse")
proto.RegisterType((*VolumeRequest)(nil), "queryrange.VolumeRequest")
proto.RegisterType((*VolumeResponse)(nil), "queryrange.VolumeResponse")
}
func init() {
@ -854,73 +854,73 @@ func init() {
}
var fileDescriptor_51b9d53b40d11902 = []byte{
// 1046 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x5d, 0x6f, 0x23, 0x35,
0x17, 0x8e, 0xf3, 0xd5, 0xc6, 0x7d, 0xb7, 0x2f, 0xb8, 0x65, 0x77, 0x54, 0x60, 0x26, 0x8a, 0x04,
0x1b, 0x24, 0x98, 0x88, 0x2e, 0xdf, 0xac, 0x10, 0x3b, 0x14, 0x44, 0xa5, 0x15, 0x42, 0xb3, 0x15,
0xf7, 0x4e, 0xc7, 0x4d, 0x86, 0xce, 0x57, 0x6d, 0xcf, 0x42, 0xef, 0xf8, 0x01, 0x20, 0xed, 0xaf,
0x58, 0x21, 0xc1, 0x2f, 0x40, 0xda, 0xfb, 0x5e, 0xf6, 0x72, 0x55, 0x89, 0x81, 0xa6, 0x37, 0x90,
0xab, 0xfe, 0x04, 0x64, 0x7b, 0x26, 0x71, 0xda, 0xb4, 0x9b, 0x74, 0x6f, 0x8a, 0xc4, 0x4d, 0x62,
0x1f, 0x9f, 0xc7, 0x3e, 0xe7, 0x39, 0xcf, 0xb1, 0x07, 0xde, 0x4e, 0x76, 0x7b, 0x9d, 0xbd, 0x94,
0x50, 0x9f, 0x50, 0xf9, 0xbf, 0x4f, 0x71, 0xd4, 0x23, 0xda, 0xd0, 0x4e, 0x68, 0xcc, 0x63, 0x04,
0xc7, 0x96, 0xb5, 0xd5, 0x5e, 0xdc, 0x8b, 0xa5, 0xb9, 0x23, 0x46, 0xca, 0x63, 0xcd, 0xea, 0xc5,
0x71, 0x2f, 0x20, 0x1d, 0x39, 0xeb, 0xa6, 0x3b, 0x1d, 0xee, 0x87, 0x84, 0x71, 0x1c, 0x26, 0xb9,
0xc3, 0xcb, 0xe2, 0xac, 0x20, 0xee, 0x29, 0x64, 0x31, 0xc8, 0x17, 0x9b, 0xf9, 0xe2, 0x5e, 0x10,
0xc6, 0x1e, 0x09, 0x3a, 0x8c, 0x63, 0xce, 0xd4, 0x6f, 0xee, 0xb1, 0x22, 0x3c, 0x92, 0x94, 0xf5,
0xe5, 0x4f, 0x6e, 0xfc, 0xec, 0x99, 0xf1, 0x77, 0x31, 0x23, 0x1d, 0x8f, 0xec, 0xf8, 0x91, 0xcf,
0xfd, 0x38, 0x62, 0xfa, 0x38, 0xdf, 0xe4, 0xbd, 0xd9, 0x36, 0x39, 0xcb, 0x49, 0xeb, 0xb0, 0x0c,
0x97, 0xee, 0xc7, 0xbb, 0xbe, 0x4b, 0xf6, 0x52, 0xc2, 0x38, 0x5a, 0x85, 0x35, 0xe9, 0x63, 0x80,
0x26, 0x68, 0x37, 0x5c, 0x35, 0x11, 0xd6, 0xc0, 0x0f, 0x7d, 0x6e, 0x94, 0x9b, 0xa0, 0x7d, 0xc3,
0x55, 0x13, 0x84, 0x60, 0x95, 0x71, 0x92, 0x18, 0x95, 0x26, 0x68, 0x57, 0x5c, 0x39, 0x46, 0x6b,
0x70, 0xd1, 0x8f, 0x38, 0xa1, 0x0f, 0x71, 0x60, 0x34, 0xa4, 0x7d, 0x34, 0x47, 0x9f, 0xc0, 0x05,
0xc6, 0x31, 0xe5, 0x5b, 0xcc, 0xa8, 0x36, 0x41, 0x7b, 0x69, 0x7d, 0xcd, 0x56, 0x7c, 0xdb, 0x05,
0xdf, 0xf6, 0x56, 0xc1, 0xb7, 0xb3, 0x78, 0x90, 0x59, 0xa5, 0x47, 0x7f, 0x58, 0xc0, 0x2d, 0x40,
0xe8, 0x23, 0x58, 0x23, 0x91, 0xb7, 0xc5, 0x8c, 0xda, 0x1c, 0x68, 0x05, 0x41, 0x6f, 0xc3, 0x86,
0xe7, 0x53, 0xb2, 0x2d, 0x38, 0x33, 0xea, 0x4d, 0xd0, 0x5e, 0x5e, 0x5f, 0xb1, 0x47, 0xf5, 0xdb,
0x28, 0x96, 0xdc, 0xb1, 0x97, 0x48, 0x2f, 0xc1, 0xbc, 0x6f, 0x2c, 0x48, 0x26, 0xe4, 0x18, 0xb5,
0x60, 0x9d, 0xf5, 0x31, 0xf5, 0x98, 0xb1, 0xd8, 0xac, 0xb4, 0x1b, 0x0e, 0x1c, 0x66, 0x56, 0x6e,
0x71, 0xf3, 0xff, 0xd6, 0xdf, 0x00, 0x22, 0x41, 0xe9, 0x66, 0xc4, 0x38, 0x8e, 0xf8, 0x55, 0x98,
0xbd, 0x0b, 0xeb, 0x42, 0x79, 0x5b, 0x4c, 0x72, 0x3b, 0x6b, 0xaa, 0x39, 0x66, 0x32, 0xd7, 0xea,
0x5c, 0xb9, 0xd6, 0xa6, 0xe6, 0x5a, 0xbf, 0x30, 0xd7, 0x5f, 0xaa, 0xf0, 0x7f, 0x4a, 0x3e, 0x2c,
0x89, 0x23, 0x46, 0x04, 0xe8, 0x01, 0xc7, 0x3c, 0x65, 0x2a, 0xcd, 0x1c, 0x24, 0x2d, 0x6e, 0xbe,
0x82, 0x3e, 0x85, 0xd5, 0x0d, 0xcc, 0xb1, 0x4c, 0x79, 0x69, 0x7d, 0xd5, 0xd6, 0x44, 0x29, 0xf6,
0x12, 0x6b, 0xce, 0x4d, 0x91, 0xd5, 0x30, 0xb3, 0x96, 0x3d, 0xcc, 0xf1, 0x9b, 0x71, 0xe8, 0x73,
0x12, 0x26, 0x7c, 0xdf, 0x95, 0x48, 0xf4, 0x2e, 0x6c, 0x7c, 0x4e, 0x69, 0x4c, 0xb7, 0xf6, 0x13,
0x22, 0x29, 0x6a, 0x38, 0xb7, 0x86, 0x99, 0xb5, 0x42, 0x0a, 0xa3, 0x86, 0x18, 0x7b, 0xa2, 0x37,
0x60, 0x4d, 0x4e, 0x24, 0x29, 0x0d, 0x67, 0x65, 0x98, 0x59, 0xff, 0x97, 0x10, 0xcd, 0x5d, 0x79,
0x4c, 0x72, 0x58, 0x9b, 0x89, 0xc3, 0x51, 0x29, 0xeb, 0x7a, 0x29, 0x0d, 0xb8, 0xf0, 0x90, 0x50,
0x26, 0xb6, 0x59, 0x90, 0xf6, 0x62, 0x8a, 0xee, 0x41, 0x28, 0x88, 0xf1, 0x19, 0xf7, 0xb7, 0x85,
0x9e, 0x04, 0x19, 0x37, 0x6c, 0x75, 0x5d, 0xb8, 0x84, 0xa5, 0x01, 0x77, 0x50, 0xce, 0x82, 0xe6,
0xe8, 0x6a, 0x63, 0xf4, 0x2b, 0x80, 0x0b, 0x5f, 0x12, 0xec, 0x11, 0xca, 0x8c, 0x46, 0xb3, 0xd2,
0x5e, 0x5a, 0x7f, 0xcd, 0xd6, 0xef, 0x86, 0xaf, 0x69, 0x1c, 0x12, 0xde, 0x27, 0x29, 0x2b, 0x0a,
0xa4, 0xbc, 0x9d, 0xdd, 0xa3, 0xcc, 0xea, 0xf6, 0x7c, 0xde, 0x4f, 0xbb, 0xf6, 0x76, 0x1c, 0x76,
0x7a, 0x14, 0xef, 0xe0, 0x08, 0x77, 0x82, 0x78, 0xd7, 0xef, 0xcc, 0x7d, 0x1f, 0x5d, 0x78, 0xce,
0x30, 0xb3, 0xc0, 0x5b, 0x6e, 0x11, 0x62, 0xeb, 0x77, 0x00, 0x5f, 0x14, 0x15, 0x7e, 0x20, 0xf6,
0x66, 0x5a, 0x63, 0x84, 0x98, 0x6f, 0xf7, 0x0d, 0x20, 0x64, 0xe6, 0xaa, 0x89, 0x7e, 0x59, 0x94,
0x9f, 0xeb, 0xb2, 0xa8, 0xcc, 0x7f, 0x59, 0x14, 0xdd, 0x50, 0x9d, 0xda, 0x0d, 0xb5, 0x0b, 0xbb,
0xe1, 0xc7, 0x8a, 0xea, 0xfc, 0x22, 0xbf, 0x39, 0x7a, 0xe2, 0x8b, 0x51, 0x4f, 0x54, 0x64, 0xb4,
0x23, 0xa9, 0xa9, 0xbd, 0x36, 0x3d, 0x12, 0x71, 0x7f, 0xc7, 0x27, 0xf4, 0x19, 0x9d, 0xa1, 0xc9,
0xad, 0x32, 0x29, 0x37, 0x5d, 0x2b, 0xd5, 0x6b, 0xaf, 0x95, 0x33, 0xdd, 0x51, 0xbb, 0x42, 0x77,
0xb4, 0x9e, 0x00, 0xf8, 0x92, 0x28, 0xc7, 0x7d, 0xdc, 0x25, 0xc1, 0x57, 0x38, 0x1c, 0x4b, 0x4e,
0x13, 0x17, 0x78, 0x2e, 0x71, 0x95, 0xaf, 0x2e, 0xae, 0x8a, 0x26, 0xae, 0xd1, 0xdb, 0x50, 0xd5,
0xde, 0x86, 0xd6, 0x69, 0x19, 0xde, 0x3c, 0x1b, 0xff, 0x1c, 0x92, 0x7a, 0x5d, 0x93, 0x54, 0xc3,
0x41, 0xff, 0x49, 0x66, 0x06, 0xc9, 0x3c, 0x06, 0x70, 0xb1, 0x78, 0x83, 0x90, 0x0d, 0xa1, 0x82,
0xc9, 0x67, 0x46, 0x11, 0xbd, 0x2c, 0xc0, 0x74, 0x64, 0x75, 0x35, 0x0f, 0xf4, 0x2d, 0xac, 0xab,
0x59, 0xde, 0xc5, 0xb7, 0xb4, 0x2e, 0xe6, 0x94, 0xe0, 0xf0, 0x9e, 0x87, 0x13, 0x4e, 0xa8, 0xf3,
0xa1, 0x88, 0xe2, 0x28, 0xb3, 0x6e, 0x5f, 0x46, 0x91, 0xfc, 0x42, 0x54, 0x38, 0x51, 0x5c, 0x75,
0xa6, 0x9b, 0x9f, 0xd0, 0xfa, 0x09, 0xc0, 0x17, 0x44, 0xa0, 0x82, 0x9a, 0x91, 0x2a, 0x36, 0xe0,
0x22, 0xcd, 0xc7, 0xb9, 0xae, 0x5b, 0xf6, 0x24, 0xad, 0x53, 0xa8, 0x74, 0xaa, 0x07, 0x99, 0x05,
0xdc, 0x11, 0x12, 0xdd, 0x99, 0xa0, 0xb1, 0x3c, 0x8d, 0x46, 0x01, 0x29, 0x4d, 0x10, 0xf7, 0x5b,
0x19, 0xa2, 0xcd, 0xc8, 0x23, 0xdf, 0x0b, 0xf1, 0x8d, 0x75, 0x9a, 0x9e, 0x8b, 0xe8, 0x95, 0x31,
0x29, 0xe7, 0xfd, 0x9d, 0x8f, 0x8f, 0x32, 0xeb, 0xfd, 0xcb, 0x58, 0xb9, 0x04, 0xac, 0xa5, 0xa0,
0x0b, 0xb7, 0x7c, 0xfd, 0xdf, 0xc5, 0xc7, 0xe2, 0x8b, 0x51, 0x34, 0xf9, 0x37, 0x71, 0x90, 0x86,
0xe4, 0xda, 0x3e, 0x8c, 0xad, 0x27, 0x65, 0xb8, 0x32, 0x11, 0x68, 0xce, 0xf7, 0x77, 0xe7, 0xca,
0xfc, 0xea, 0xb8, 0xcc, 0x53, 0x00, 0xce, 0xdd, 0xa3, 0xcc, 0xfa, 0x60, 0xa6, 0x3a, 0x4f, 0x41,
0xff, 0x6b, 0x0b, 0xed, 0xbc, 0x73, 0x78, 0x6c, 0x96, 0x9e, 0x1e, 0x9b, 0xa5, 0xd3, 0x63, 0x13,
0xfc, 0x30, 0x30, 0xc1, 0xcf, 0x03, 0x13, 0x1c, 0x0c, 0x4c, 0x70, 0x38, 0x30, 0xc1, 0x9f, 0x03,
0x13, 0xfc, 0x35, 0x30, 0x4b, 0xa7, 0x03, 0x13, 0x3c, 0x3a, 0x31, 0x4b, 0x87, 0x27, 0x66, 0xe9,
0xe9, 0x89, 0x59, 0xea, 0xd6, 0x25, 0x13, 0x77, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x8e, 0x54,
0x1e, 0x08, 0xe9, 0x0e, 0x00, 0x00,
// 1042 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x4d, 0x6f, 0x23, 0x35,
0x18, 0x8e, 0xf3, 0xd5, 0xc6, 0x65, 0x0b, 0xb8, 0x65, 0x77, 0x54, 0xd0, 0x4c, 0x14, 0x09, 0x36,
0x48, 0x30, 0x11, 0x2d, 0xb0, 0xe2, 0x43, 0x88, 0x1d, 0x0a, 0xa2, 0xd2, 0x0a, 0xa1, 0xd9, 0x88,
0xbb, 0xd3, 0x71, 0x93, 0xa1, 0xf3, 0x55, 0xdb, 0xb3, 0xa2, 0x37, 0x7e, 0x00, 0x48, 0xfb, 0x17,
0xb8, 0x20, 0x24, 0x10, 0x3f, 0x00, 0x89, 0x7b, 0x8f, 0x3d, 0xae, 0x2a, 0x31, 0xd0, 0xf4, 0x02,
0x39, 0xf5, 0x27, 0x20, 0xdb, 0x33, 0x89, 0xd3, 0xaf, 0x4d, 0xba, 0x97, 0x22, 0x71, 0xc9, 0xd8,
0xaf, 0xdf, 0xc7, 0x7e, 0xdf, 0xe7, 0x7d, 0x5e, 0x3b, 0xf0, 0x6e, 0xb2, 0xdb, 0xef, 0xec, 0xa5,
0x84, 0xfa, 0x84, 0xca, 0xef, 0x3e, 0xc5, 0x51, 0x9f, 0x68, 0x43, 0x3b, 0xa1, 0x31, 0x8f, 0x11,
0x9c, 0x58, 0xd6, 0x56, 0xfb, 0x71, 0x3f, 0x96, 0xe6, 0x8e, 0x18, 0x29, 0x8f, 0x35, 0xab, 0x1f,
0xc7, 0xfd, 0x80, 0x74, 0xe4, 0xac, 0x97, 0xee, 0x74, 0xb8, 0x1f, 0x12, 0xc6, 0x71, 0x98, 0xe4,
0x0e, 0x2f, 0x8b, 0xb3, 0x82, 0xb8, 0xaf, 0x90, 0xc5, 0x20, 0x5f, 0x6c, 0xe6, 0x8b, 0x7b, 0x41,
0x18, 0x7b, 0x24, 0xe8, 0x30, 0x8e, 0x39, 0x53, 0xbf, 0xb9, 0xc7, 0x8a, 0xf0, 0x48, 0x52, 0x36,
0x90, 0x3f, 0xb9, 0xf1, 0x93, 0xa7, 0xc6, 0xdf, 0xc3, 0x8c, 0x74, 0x3c, 0xb2, 0xe3, 0x47, 0x3e,
0xf7, 0xe3, 0x88, 0xe9, 0xe3, 0x7c, 0x93, 0x77, 0x67, 0xdb, 0xe4, 0x2c, 0x27, 0xad, 0xc3, 0x32,
0x5c, 0x7a, 0x10, 0xef, 0xfa, 0x2e, 0xd9, 0x4b, 0x09, 0xe3, 0x68, 0x15, 0xd6, 0xa4, 0x8f, 0x01,
0x9a, 0xa0, 0xdd, 0x70, 0xd5, 0x44, 0x58, 0x03, 0x3f, 0xf4, 0xb9, 0x51, 0x6e, 0x82, 0xf6, 0x2d,
0x57, 0x4d, 0x10, 0x82, 0x55, 0xc6, 0x49, 0x62, 0x54, 0x9a, 0xa0, 0x5d, 0x71, 0xe5, 0x18, 0xad,
0xc1, 0x45, 0x3f, 0xe2, 0x84, 0x3e, 0xc2, 0x81, 0xd1, 0x90, 0xf6, 0xf1, 0x1c, 0x7d, 0x04, 0x17,
0x18, 0xc7, 0x94, 0x77, 0x99, 0x51, 0x6d, 0x82, 0xf6, 0xd2, 0xfa, 0x9a, 0xad, 0xf8, 0xb6, 0x0b,
0xbe, 0xed, 0x6e, 0xc1, 0xb7, 0xb3, 0x78, 0x90, 0x59, 0xa5, 0xc7, 0x7f, 0x5a, 0xc0, 0x2d, 0x40,
0xe8, 0x7d, 0x58, 0x23, 0x91, 0xd7, 0x65, 0x46, 0x6d, 0x0e, 0xb4, 0x82, 0xa0, 0xb7, 0x60, 0xc3,
0xf3, 0x29, 0xd9, 0x16, 0x9c, 0x19, 0xf5, 0x26, 0x68, 0x2f, 0xaf, 0xaf, 0xd8, 0xe3, 0xfa, 0x6d,
0x16, 0x4b, 0xee, 0xc4, 0x4b, 0xa4, 0x97, 0x60, 0x3e, 0x30, 0x16, 0x24, 0x13, 0x72, 0x8c, 0x5a,
0xb0, 0xce, 0x06, 0x98, 0x7a, 0xcc, 0x58, 0x6c, 0x56, 0xda, 0x0d, 0x07, 0x8e, 0x32, 0x2b, 0xb7,
0xb8, 0xf9, 0xb7, 0xf5, 0x0f, 0x80, 0x48, 0x50, 0xba, 0x15, 0x31, 0x8e, 0x23, 0x7e, 0x1d, 0x66,
0x3f, 0x84, 0x75, 0xa1, 0xbc, 0x2e, 0x93, 0xdc, 0xce, 0x9a, 0x6a, 0x8e, 0x99, 0xce, 0xb5, 0x3a,
0x57, 0xae, 0xb5, 0x0b, 0x73, 0xad, 0x5f, 0x9a, 0xeb, 0xcf, 0x55, 0xf8, 0x9c, 0x92, 0x0f, 0x4b,
0xe2, 0x88, 0x11, 0x01, 0x7a, 0xc8, 0x31, 0x4f, 0x99, 0x4a, 0x33, 0x07, 0x49, 0x8b, 0x9b, 0xaf,
0xa0, 0x8f, 0x61, 0x75, 0x13, 0x73, 0x2c, 0x53, 0x5e, 0x5a, 0x5f, 0xb5, 0x35, 0x51, 0x8a, 0xbd,
0xc4, 0x9a, 0x73, 0x5b, 0x64, 0x35, 0xca, 0xac, 0x65, 0x0f, 0x73, 0xfc, 0x46, 0x1c, 0xfa, 0x9c,
0x84, 0x09, 0xdf, 0x77, 0x25, 0x12, 0xbd, 0x03, 0x1b, 0x9f, 0x52, 0x1a, 0xd3, 0xee, 0x7e, 0x42,
0x24, 0x45, 0x0d, 0xe7, 0xce, 0x28, 0xb3, 0x56, 0x48, 0x61, 0xd4, 0x10, 0x13, 0x4f, 0xf4, 0x3a,
0xac, 0xc9, 0x89, 0x24, 0xa5, 0xe1, 0xac, 0x8c, 0x32, 0xeb, 0x79, 0x09, 0xd1, 0xdc, 0x95, 0xc7,
0x34, 0x87, 0xb5, 0x99, 0x38, 0x1c, 0x97, 0xb2, 0xae, 0x97, 0xd2, 0x80, 0x0b, 0x8f, 0x08, 0x65,
0x62, 0x9b, 0x05, 0x69, 0x2f, 0xa6, 0xe8, 0x3e, 0x84, 0x82, 0x18, 0x9f, 0x71, 0x7f, 0x5b, 0xe8,
0x49, 0x90, 0x71, 0xcb, 0x56, 0xd7, 0x85, 0x4b, 0x58, 0x1a, 0x70, 0x07, 0xe5, 0x2c, 0x68, 0x8e,
0xae, 0x36, 0x46, 0xbf, 0x00, 0xb8, 0xf0, 0x39, 0xc1, 0x1e, 0xa1, 0xcc, 0x68, 0x34, 0x2b, 0xed,
0xa5, 0xf5, 0x57, 0x6d, 0xfd, 0x6e, 0xf8, 0x92, 0xc6, 0x21, 0xe1, 0x03, 0x92, 0xb2, 0xa2, 0x40,
0xca, 0xdb, 0xd9, 0x3d, 0xca, 0xac, 0x5e, 0xdf, 0xe7, 0x83, 0xb4, 0x67, 0x6f, 0xc7, 0x61, 0xa7,
0x4f, 0xf1, 0x0e, 0x8e, 0x70, 0x27, 0x88, 0x77, 0xfd, 0xce, 0xdc, 0xf7, 0xd1, 0xa5, 0xe7, 0x8c,
0x32, 0x0b, 0xbc, 0xe9, 0x16, 0x21, 0xb6, 0xfe, 0x00, 0xf0, 0x45, 0x51, 0xe1, 0x87, 0x62, 0x6f,
0xa6, 0x35, 0x46, 0x88, 0xf9, 0xf6, 0xc0, 0x00, 0x42, 0x66, 0xae, 0x9a, 0xe8, 0x97, 0x45, 0xf9,
0x99, 0x2e, 0x8b, 0xca, 0xfc, 0x97, 0x45, 0xd1, 0x0d, 0xd5, 0x0b, 0xbb, 0xa1, 0x76, 0x69, 0x37,
0x7c, 0x57, 0x51, 0x9d, 0x5f, 0xe4, 0x37, 0x47, 0x4f, 0x7c, 0x36, 0xee, 0x89, 0x8a, 0x8c, 0x76,
0x2c, 0x35, 0xb5, 0xd7, 0x96, 0x47, 0x22, 0xee, 0xef, 0xf8, 0x84, 0x3e, 0xa5, 0x33, 0x34, 0xb9,
0x55, 0xa6, 0xe5, 0xa6, 0x6b, 0xa5, 0x7a, 0xe3, 0xb5, 0x72, 0xa6, 0x3b, 0x6a, 0xd7, 0xe8, 0x8e,
0xd6, 0xef, 0x00, 0xbe, 0x24, 0xca, 0xf1, 0x00, 0xf7, 0x48, 0xf0, 0x05, 0x0e, 0x27, 0x92, 0xd3,
0xc4, 0x05, 0x9e, 0x49, 0x5c, 0xe5, 0xeb, 0x8b, 0xab, 0xa2, 0x89, 0x6b, 0xfc, 0x36, 0x54, 0xb5,
0xb7, 0xa1, 0x75, 0x5a, 0x86, 0xb7, 0xcf, 0xc6, 0x3f, 0x87, 0xa4, 0x5e, 0xd3, 0x24, 0xd5, 0x70,
0xd0, 0xff, 0x92, 0x99, 0x41, 0x32, 0x3f, 0x02, 0xb8, 0x58, 0xbc, 0x41, 0xc8, 0x86, 0x50, 0xc1,
0xe4, 0x33, 0xa3, 0x88, 0x5e, 0x16, 0x60, 0x3a, 0xb6, 0xba, 0x9a, 0x07, 0xfa, 0x1a, 0xd6, 0xd5,
0x2c, 0xef, 0xe2, 0x3b, 0x5a, 0x17, 0x73, 0x4a, 0x70, 0x78, 0xdf, 0xc3, 0x09, 0x27, 0xd4, 0x79,
0x4f, 0x44, 0x71, 0x94, 0x59, 0x77, 0xaf, 0xa2, 0x48, 0xfe, 0x43, 0x54, 0x38, 0x51, 0x5c, 0x75,
0xa6, 0x9b, 0x9f, 0xd0, 0xfa, 0x1e, 0xc0, 0x17, 0x44, 0xa0, 0x82, 0x9a, 0xb1, 0x2a, 0x36, 0xe1,
0x22, 0xcd, 0xc7, 0xb9, 0xae, 0x5b, 0xf6, 0x34, 0xad, 0x17, 0x50, 0xe9, 0x54, 0x0f, 0x32, 0x0b,
0xb8, 0x63, 0x24, 0xda, 0x98, 0xa2, 0xb1, 0x7c, 0x11, 0x8d, 0x02, 0x52, 0x9a, 0x22, 0xee, 0xb7,
0x32, 0x44, 0x5b, 0x91, 0x47, 0xbe, 0x11, 0xe2, 0x9b, 0xe8, 0x34, 0x3d, 0x17, 0xd1, 0x2b, 0x13,
0x52, 0xce, 0xfb, 0x3b, 0x1f, 0x1c, 0x65, 0xd6, 0xbd, 0xab, 0x58, 0xb9, 0x02, 0xac, 0xa5, 0xa0,
0x0b, 0xb7, 0x7c, 0xf3, 0xdf, 0xc5, 0x1f, 0x00, 0xbc, 0xf5, 0x55, 0x1c, 0xa4, 0x21, 0xb9, 0xb1,
0x6f, 0x62, 0xeb, 0xd7, 0x32, 0x5c, 0x2e, 0x62, 0xcc, 0x59, 0x0e, 0xcf, 0x15, 0xd7, 0x98, 0x14,
0x77, 0xda, 0xd7, 0xb9, 0x77, 0x94, 0x59, 0x1b, 0x33, 0x15, 0x76, 0x1a, 0xf8, 0x9f, 0x2d, 0xaa,
0xf3, 0xf6, 0xe1, 0xb1, 0x59, 0x7a, 0x72, 0x6c, 0x96, 0x4e, 0x8f, 0x4d, 0xf0, 0xed, 0xd0, 0x04,
0x3f, 0x0d, 0x4d, 0x70, 0x30, 0x34, 0xc1, 0xe1, 0xd0, 0x04, 0x7f, 0x0d, 0x4d, 0xf0, 0xf7, 0xd0,
0x2c, 0x9d, 0x0e, 0x4d, 0xf0, 0xf8, 0xc4, 0x2c, 0x1d, 0x9e, 0x98, 0xa5, 0x27, 0x27, 0x66, 0xa9,
0x57, 0x97, 0x24, 0x6c, 0xfc, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x0d, 0x3e, 0xc1, 0x91, 0xd5, 0x0e,
0x00, 0x00,
}
func (this *LokiRequest) Equal(that interface{}) bool {
@ -1339,14 +1339,14 @@ func (this *IndexStatsResponse) Equal(that interface{}) bool {
}
return true
}
func (this *LabelVolumeRequest) Equal(that interface{}) bool {
func (this *VolumeRequest) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LabelVolumeRequest)
that1, ok := that.(*VolumeRequest)
if !ok {
that2, ok := that.(LabelVolumeRequest)
that2, ok := that.(VolumeRequest)
if ok {
that1 = &that2
} else {
@ -1374,14 +1374,14 @@ func (this *LabelVolumeRequest) Equal(that interface{}) bool {
}
return true
}
func (this *LabelVolumeResponse) Equal(that interface{}) bool {
func (this *VolumeResponse) Equal(that interface{}) bool {
if that == nil {
return this == nil
}
that1, ok := that.(*LabelVolumeResponse)
that1, ok := that.(*VolumeResponse)
if !ok {
that2, ok := that.(LabelVolumeResponse)
that2, ok := that.(VolumeResponse)
if ok {
that1 = &that2
} else {
@ -1557,24 +1557,24 @@ func (this *IndexStatsResponse) GoString() string {
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LabelVolumeRequest) GoString() string {
func (this *VolumeRequest) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 7)
s = append(s, "&queryrange.LabelVolumeRequest{")
s = append(s, "&queryrange.VolumeRequest{")
s = append(s, "Match: "+fmt.Sprintf("%#v", this.Match)+",\n")
s = append(s, "StartTs: "+fmt.Sprintf("%#v", this.StartTs)+",\n")
s = append(s, "EndTs: "+fmt.Sprintf("%#v", this.EndTs)+",\n")
s = append(s, "}")
return strings.Join(s, "")
}
func (this *LabelVolumeResponse) GoString() string {
func (this *VolumeResponse) GoString() string {
if this == nil {
return "nil"
}
s := make([]string, 0, 6)
s = append(s, "&queryrange.LabelVolumeResponse{")
s = append(s, "&queryrange.VolumeResponse{")
s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n")
s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n")
s = append(s, "}")
@ -2223,7 +2223,7 @@ func (m *IndexStatsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LabelVolumeRequest) Marshal() (dAtA []byte, err error) {
func (m *VolumeRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
@ -2233,12 +2233,12 @@ func (m *LabelVolumeRequest) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
func (m *LabelVolumeRequest) MarshalTo(dAtA []byte) (int, error) {
func (m *VolumeRequest) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LabelVolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
func (m *VolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
@ -2271,7 +2271,7 @@ func (m *LabelVolumeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
func (m *LabelVolumeResponse) Marshal() (dAtA []byte, err error) {
func (m *VolumeResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
@ -2281,12 +2281,12 @@ func (m *LabelVolumeResponse) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
func (m *LabelVolumeResponse) MarshalTo(dAtA []byte) (int, error) {
func (m *VolumeResponse) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *LabelVolumeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
func (m *VolumeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
@ -2604,7 +2604,7 @@ func (m *IndexStatsResponse) Size() (n int) {
return n
}
func (m *LabelVolumeRequest) Size() (n int) {
func (m *VolumeRequest) Size() (n int) {
if m == nil {
return 0
}
@ -2623,7 +2623,7 @@ func (m *LabelVolumeRequest) Size() (n int) {
return n
}
func (m *LabelVolumeResponse) Size() (n int) {
func (m *VolumeResponse) Size() (n int) {
if m == nil {
return 0
}
@ -2792,11 +2792,11 @@ func (this *IndexStatsResponse) String() string {
}, "")
return s
}
func (this *LabelVolumeRequest) String() string {
func (this *VolumeRequest) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LabelVolumeRequest{`,
s := strings.Join([]string{`&VolumeRequest{`,
`Match:` + fmt.Sprintf("%v", this.Match) + `,`,
`StartTs:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.StartTs), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`,
`EndTs:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.EndTs), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`,
@ -2804,11 +2804,11 @@ func (this *LabelVolumeRequest) String() string {
}, "")
return s
}
func (this *LabelVolumeResponse) String() string {
func (this *VolumeResponse) String() string {
if this == nil {
return "nil"
}
s := strings.Join([]string{`&LabelVolumeResponse{`,
s := strings.Join([]string{`&VolumeResponse{`,
`Response:` + fmt.Sprintf("%v", this.Response) + `,`,
`Headers:` + fmt.Sprintf("%v", this.Headers) + `,`,
`}`,
@ -4810,7 +4810,7 @@ func (m *IndexStatsResponse) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
func (m *VolumeRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
@ -4833,10 +4833,10 @@ func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LabelVolumeRequest: wiretype end group for non-group")
return fmt.Errorf("proto: VolumeRequest: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LabelVolumeRequest: illegal tag %d (wire type %d)", fieldNum, wire)
return fmt.Errorf("proto: VolumeRequest: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
@ -4961,7 +4961,7 @@ func (m *LabelVolumeRequest) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
func (m *VolumeResponse) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
@ -4984,10 +4984,10 @@ func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: LabelVolumeResponse: wiretype end group for non-group")
return fmt.Errorf("proto: VolumeResponse: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: LabelVolumeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
return fmt.Errorf("proto: VolumeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
@ -5020,7 +5020,7 @@ func (m *LabelVolumeResponse) Unmarshal(dAtA []byte) error {
return io.ErrUnexpectedEOF
}
if m.Response == nil {
m.Response = &github_com_grafana_loki_pkg_logproto.LabelVolumeResponse{}
m.Response = &github_com_grafana_loki_pkg_logproto.VolumeResponse{}
}
if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err

@ -145,7 +145,7 @@ message IndexStatsResponse {
];
}
message LabelVolumeRequest {
message VolumeRequest {
repeated string match = 1;
google.protobuf.Timestamp startTs = 2 [
(gogoproto.stdtime) = true,
@ -157,8 +157,8 @@ message LabelVolumeRequest {
];
}
message LabelVolumeResponse {
logproto.LabelVolumeResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/logproto.LabelVolumeResponse"];
message VolumeResponse {
logproto.VolumeResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/logproto.VolumeResponse"];
repeated definitions.PrometheusResponseHeader Headers = 2 [
(gogoproto.jsontag) = "-",
(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader"

@ -166,7 +166,7 @@ func NewTripperware(
return nil, nil, err
}
labelVolumeTripperware, err := NewLabelVolumeTripperware(cfg, log, limits, schema, LokiCodec, statsCache, cacheGenNumLoader, retentionEnabled, metrics)
labelVolumeTripperware, err := NewSeriesVolumeTripperware(cfg, log, limits, schema, LokiCodec, statsCache, cacheGenNumLoader, retentionEnabled, metrics)
if err != nil {
return nil, nil, err
}
@ -315,12 +315,12 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
level.Info(logger).Log("msg", "executing query", "type", "stats", "query", statsQuery.Query, "length", statsQuery.End.Sub(statsQuery.Start))
return r.indexStats.RoundTrip(req)
case LabelVolumeOp:
volumeQuery, err := loghttp.ParseLabelVolumeQuery(req)
case SeriesVolumeOp:
volumeQuery, err := loghttp.ParseSeriesVolumeQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
level.Info(logger).Log("msg", "executing query", "type", "label_volume", "query", volumeQuery.Query, "length", volumeQuery.End.Sub(volumeQuery.Start), "limit", volumeQuery.Limit)
level.Info(logger).Log("msg", "executing query", "type", "series_volume", "query", volumeQuery.Query, "length", volumeQuery.End.Sub(volumeQuery.Start), "limit", volumeQuery.Limit)
return r.labelVolume.RoundTrip(req)
default:
@ -353,7 +353,7 @@ const (
SeriesOp = "series"
LabelNamesOp = "labels"
IndexStatsOp = "index_stats"
LabelVolumeOp = "label_volume"
SeriesVolumeOp = "series_volume"
)
func getOperation(path string) string {
@ -368,8 +368,8 @@ func getOperation(path string) string {
return InstantQueryOp
case path == "/loki/api/v1/index/stats":
return IndexStatsOp
case path == "/loki/api/v1/index/label_volume":
return LabelVolumeOp
case path == "/loki/api/v1/index/series_volume":
return SeriesVolumeOp
default:
return ""
}
@ -743,7 +743,7 @@ func NewInstantMetricTripperware(
}, nil
}
func NewLabelVolumeTripperware(cfg Config,
func NewSeriesVolumeTripperware(cfg Config,
log log.Logger,
limits Limits,
schema config.SchemaConfig,

@ -129,10 +129,10 @@ var (
},
}
labelVolume = logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 1024},
{Name: "bar", Value: "baz", Volume: 3350},
seriesVolume = logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 1024},
{Name: `{bar="baz"}`, Value: "", Volume: 3350},
},
Limit: 10,
}
@ -547,7 +547,7 @@ func TestIndexStatsTripperware(t *testing.T) {
require.Equal(t, response.Entries*2, res.Response.Entries)
}
func TestLabelVolumeTripperware(t *testing.T) {
func TestSeriesVolumeTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, testEngineOpts, util_log.Logger, fakeLimits{maxQueryLength: 48 * time.Hour, maxQueryParallelism: 1}, config.SchemaConfig{Configs: testSchemas}, nil, false, nil)
if stopper != nil {
defer stopper.Stop()
@ -558,7 +558,7 @@ func TestLabelVolumeTripperware(t *testing.T) {
require.NoError(t, err)
defer rt.Close()
lreq := &logproto.LabelVolumeRequest{
lreq := &logproto.VolumeRequest{
Matchers: `{job="varlogs"}`,
From: model.TimeFromUnixNano(testTime.Add(-25 * time.Hour).UnixNano()), // bigger than split by interval limit
Through: model.TimeFromUnixNano(testTime.UnixNano()),
@ -573,7 +573,7 @@ func TestLabelVolumeTripperware(t *testing.T) {
err = user.InjectOrgIDIntoHTTPRequest(ctx, req)
require.NoError(t, err)
count, h := labelVolumeResult(labelVolume)
count, h := labelVolumeResult(seriesVolume)
rt.setHandler(h)
resp, err := tpw(rt).RoundTrip(req)
@ -583,14 +583,14 @@ func TestLabelVolumeTripperware(t *testing.T) {
volumeResp, err := LokiCodec.DecodeResponse(ctx, resp, lreq)
require.NoError(t, err)
expected := logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{ // add volumes from across shards
{Name: "bar", Value: "baz", Volume: 6700},
{Name: "foo", Value: "bar", Volume: 2048},
expected := logproto.VolumeResponse{
Volumes: []logproto.Volume{ // add volumes from across shards
{Name: `{bar="baz"}`, Value: "", Volume: 6700},
{Name: `{foo="bar"}`, Value: "", Volume: 2048},
},
}
res, ok := volumeResp.(*LabelVolumeResponse)
res, ok := volumeResp.(*VolumeResponse)
require.Equal(t, true, ok)
require.Equal(t, expected.Volumes, res.Response.Volumes)
}
@ -1279,13 +1279,13 @@ func indexStatsResult(v logproto.IndexStatsResponse) (*int, http.Handler) {
})
}
func labelVolumeResult(v logproto.LabelVolumeResponse) (*int, http.Handler) {
func labelVolumeResult(v logproto.VolumeResponse) (*int, http.Handler) {
count := 0
var lock sync.Mutex
return &count, http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
lock.Lock()
defer lock.Unlock()
if err := marshal.WriteLabelVolumeResponseJSON(&v, w); err != nil {
if err := marshal.WriteSeriesVolumeResponseJSON(&v, w); err != nil {
panic(err)
}
count++

@ -213,7 +213,7 @@ func (h *splitByInterval) Do(ctx context.Context, r queryrangebase.Request) (que
intervals[i], intervals[j] = intervals[j], intervals[i]
}
}
case *LokiSeriesRequest, *LokiLabelNamesRequest, *logproto.IndexStatsRequest, *logproto.LabelVolumeRequest:
case *LokiSeriesRequest, *LokiLabelNamesRequest, *logproto.IndexStatsRequest, *logproto.VolumeRequest:
// Set this to 0 since this is not used in Series/Labels/Index Request.
limit = 0
default:
@ -290,11 +290,11 @@ func splitByTime(req queryrangebase.Request, interval time.Duration) ([]queryran
Matchers: r.GetMatchers(),
})
})
case *logproto.LabelVolumeRequest:
case *logproto.VolumeRequest:
startTS := model.Time(r.GetStart()).Time()
endTS := model.Time(r.GetEnd()).Time()
util.ForInterval(interval, startTS, endTS, true, func(start, end time.Time) {
reqs = append(reqs, &logproto.LabelVolumeRequest{
reqs = append(reqs, &logproto.VolumeRequest{
From: model.TimeFromUnix(start.Unix()),
Through: model.TimeFromUnix(end.Unix()),
Matchers: r.GetMatchers(),

@ -851,14 +851,14 @@ func Test_series_splitByInterval_Do(t *testing.T) {
}
}
func Test_labelvolume_splitByInterval_Do(t *testing.T) {
func Test_seriesvolume_splitByInterval_Do(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
next := queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
return &LabelVolumeResponse{
Response: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
{Name: "bar", Value: "baz", Volume: 28},
return &VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
{Name: `{bar="baz"}`, Value: "", Volume: 28},
},
Limit: 1},
Headers: nil,
@ -876,21 +876,21 @@ func Test_labelvolume_splitByInterval_Do(t *testing.T) {
tests := []struct {
name string
req *logproto.LabelVolumeRequest
want *LabelVolumeResponse
req *logproto.VolumeRequest
want *VolumeResponse
}{
{
"label volumes",
&logproto.LabelVolumeRequest{
&logproto.VolumeRequest{
From: model.TimeFromUnixNano(start.UnixNano()),
Through: model.TimeFromUnixNano(end.UnixNano()),
Matchers: "{}",
Limit: 1,
},
&LabelVolumeResponse{
Response: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 76},
&VolumeResponse{
Response: &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 76},
},
Limit: 1,
},

@ -5,8 +5,10 @@ import (
"fmt"
"time"
"github.com/opentracing/opentracing-go"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/concurrency"
@ -25,7 +27,7 @@ import (
type IngesterQuerier interface {
GetChunkIDs(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([]string, error)
Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error)
LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error)
SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error)
}
type AsyncStoreCfg struct {
@ -162,15 +164,18 @@ func (a *AsyncStore) Stats(ctx context.Context, userID string, from, through mod
return &merged, nil
}
func (a *AsyncStore) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (a *AsyncStore) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "AsyncStore.SeriesVolume")
defer sp.Finish()
logger := util_log.WithContext(ctx, util_log.Logger)
matchersStr := syntax.MatchersString(matchers)
type f func() (*logproto.LabelVolumeResponse, error)
type f func() (*logproto.VolumeResponse, error)
var jobs []f
if a.shouldQueryIngesters(through, model.Now()) {
jobs = append(jobs, func() (*logproto.LabelVolumeResponse, error) {
vols, err := a.ingesterQuerier.LabelVolume(ctx, userID, from, through, limit, matchers...)
jobs = append(jobs, func() (*logproto.VolumeResponse, error) {
vols, err := a.ingesterQuerier.SeriesVolume(ctx, userID, from, through, limit, matchers...)
level.Debug(logger).Log(
"msg", "queried label volumes",
"matchers", matchersStr,
@ -179,8 +184,8 @@ func (a *AsyncStore) LabelVolume(ctx context.Context, userID string, from, throu
return vols, err
})
}
jobs = append(jobs, func() (*logproto.LabelVolumeResponse, error) {
vols, err := a.Store.LabelVolume(ctx, userID, from, through, limit, matchers...)
jobs = append(jobs, func() (*logproto.VolumeResponse, error) {
vols, err := a.Store.SeriesVolume(ctx, userID, from, through, limit, matchers...)
level.Debug(logger).Log(
"msg", "queried label volume",
"matchers", matchersStr,
@ -189,7 +194,7 @@ func (a *AsyncStore) LabelVolume(ctx context.Context, userID string, from, throu
return vols, err
})
resps := make([]*logproto.LabelVolumeResponse, len(jobs))
resps := make([]*logproto.VolumeResponse, len(jobs))
if err := concurrency.ForEachJob(
ctx,
len(jobs),
@ -203,7 +208,15 @@ func (a *AsyncStore) LabelVolume(ctx context.Context, userID string, from, throu
return nil, err
}
merged := labelvolume.Merge(resps, limit)
sp.LogKV(
"user", userID,
"from", from.Time(),
"through", through.Time(),
"matchers", syntax.MatchersString(matchers),
"limit", limit,
)
merged := seriesvolume.Merge(resps, limit)
return merged, nil
}

@ -39,14 +39,14 @@ func (s *storeMock) GetChunkFetcher(tm model.Time) *fetcher.Fetcher {
return args.Get(0).(*fetcher.Fetcher)
}
func (s *storeMock) LabelVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (s *storeMock) SeriesVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
args := s.Called(userID, from, through, matchers)
if args.Get(0) == nil {
return nil, args.Error(1)
}
return args.Get(0).(*logproto.LabelVolumeResponse), args.Error(1)
return args.Get(0).(*logproto.VolumeResponse), args.Error(1)
}
type ingesterQuerierMock struct {
@ -63,14 +63,14 @@ func (i *ingesterQuerierMock) GetChunkIDs(ctx context.Context, from, through mod
return args.Get(0).([]string), args.Error(1)
}
func (i *ingesterQuerierMock) LabelVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (i *ingesterQuerierMock) SeriesVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
args := i.Called(userID, from, through, matchers)
if args.Get(0) == nil {
return nil, args.Error(1)
}
return args.Get(0).(*logproto.LabelVolumeResponse), args.Error(1)
return args.Get(0).(*logproto.VolumeResponse), args.Error(1)
}
func buildMockChunkRef(t *testing.T, num int) []chunk.Chunk {
@ -310,20 +310,20 @@ func TestAsyncStore_QueryIngestersWithin(t *testing.T) {
}
}
func TestLabelVolume(t *testing.T) {
func TestSeriesVolume(t *testing.T) {
store := newStoreMock()
store.On("LabelVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(
&logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 38},
store.On("SeriesVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(
&logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
Limit: 10,
}, nil)
ingesterQuerier := newIngesterQuerierMock()
ingesterQuerier.On("LabelVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "bar", Value: "baz", Volume: 38},
ingesterQuerier.On("SeriesVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{bar="baz"}`, Value: "", Volume: 38},
},
Limit: 10,
}, nil)
@ -334,13 +334,13 @@ func TestLabelVolume(t *testing.T) {
}
asyncStore := NewAsyncStore(asyncStoreCfg, store, config.SchemaConfig{})
vol, err := asyncStore.LabelVolume(context.Background(), "test", model.Now().Add(-2*time.Hour), model.Now(), 10, nil...)
vol, err := asyncStore.SeriesVolume(context.Background(), "test", model.Now().Add(-2*time.Hour), model.Now(), 10, nil...)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "bar", Value: "baz", Volume: 38},
{Name: "foo", Value: "bar", Volume: 38},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{bar="baz"}`, Value: "", Volume: 38},
{Name: `{foo="bar"}`, Value: "", Volume: 38},
},
Limit: 10,
}, vol)

@ -4,7 +4,7 @@ import (
"context"
"sort"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
@ -195,10 +195,10 @@ func (c compositeStore) Stats(ctx context.Context, userID string, from, through
return &res, err
}
func (c compositeStore) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
volumes := make([]*logproto.LabelVolumeResponse, 0, len(c.stores))
func (c compositeStore) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
volumes := make([]*logproto.VolumeResponse, 0, len(c.stores))
err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error {
volume, err := store.LabelVolume(innerCtx, userID, from, through, limit, matchers...)
volume, err := store.SeriesVolume(innerCtx, userID, from, through, limit, matchers...)
volumes = append(volumes, volume)
return err
})
@ -207,7 +207,7 @@ func (c compositeStore) LabelVolume(ctx context.Context, userID string, from, th
return nil, err
}
res := labelvolume.Merge(volumes, limit)
res := seriesvolume.Merge(volumes, limit)
return res, err
}

@ -5,6 +5,8 @@ import (
"fmt"
"time"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logproto"
"github.com/go-kit/log/level"
@ -130,8 +132,8 @@ func (c *storeEntry) Stats(ctx context.Context, userID string, from, through mod
return c.indexReader.Stats(ctx, userID, from, through, matchers...)
}
func (c *storeEntry) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelVolume")
func (c *storeEntry) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.Volume")
defer sp.Finish()
shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through)
@ -141,7 +143,16 @@ func (c *storeEntry) LabelVolume(ctx context.Context, userID string, from, throu
return nil, nil
}
return c.indexReader.LabelVolume(ctx, userID, from, through, limit, matchers...)
sp.LogKV(
"user", userID,
"from", from.Time(),
"through", through.Time(),
"matchers", syntax.MatchersString(matchers),
"err", err,
"limit", limit,
)
return c.indexReader.SeriesVolume(ctx, userID, from, through, limit, matchers...)
}
func (c *storeEntry) validateQueryTimeRange(ctx context.Context, userID string, from *model.Time, through *model.Time) (bool, error) {

@ -56,7 +56,7 @@ func (m mockStore) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*lab
return nil, nil
}
func (m mockStore) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (m mockStore) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return nil, nil
}
@ -299,45 +299,45 @@ func TestCompositeStore_GetChunkFetcher(t *testing.T) {
}
}
type mockStoreLabelVolume struct {
type mockStoreSeriesVolume struct {
mockStore
value *logproto.LabelVolumeResponse
value *logproto.VolumeResponse
err error
}
func (m mockStoreLabelVolume) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (m mockStoreSeriesVolume) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return m.value, m.err
}
func TestLabelVolume(t *testing.T) {
func TestSeriesVolume(t *testing.T) {
t.Run("it returns volumes from all stores", func(t *testing.T) {
cs := compositeStore{
stores: []compositeStoreEntry{
{model.TimeFromUnix(10), mockStoreLabelVolume{mockStore: mockStore(0), value: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{{Name: "foo", Value: "bar", Volume: 15}}, Limit: 10,
{model.TimeFromUnix(10), mockStoreSeriesVolume{mockStore: mockStore(0), value: &logproto.VolumeResponse{
Volumes: []logproto.Volume{{Name: `{foo="bar"}`, Value: "", Volume: 15}}, Limit: 10,
}}},
{model.TimeFromUnix(20), mockStoreLabelVolume{mockStore: mockStore(1), value: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{{Name: "foo", Value: "bar", Volume: 30}}, Limit: 10,
{model.TimeFromUnix(20), mockStoreSeriesVolume{mockStore: mockStore(1), value: &logproto.VolumeResponse{
Volumes: []logproto.Volume{{Name: `{foo="bar"}`, Value: "", Volume: 30}}, Limit: 10,
}}},
},
}
volumes, err := cs.LabelVolume(context.Background(), "fake", 10001, 20001, 10, nil)
volumes, err := cs.SeriesVolume(context.Background(), "fake", 10001, 20001, 10, nil)
require.NoError(t, err)
require.Equal(t, []logproto.LabelVolume{{Name: "foo", Value: "bar", Volume: 45}}, volumes.Volumes)
require.Equal(t, []logproto.Volume{{Name: `{foo="bar"}`, Value: "", Volume: 45}}, volumes.Volumes)
})
t.Run("it returns an error if any store returns an error", func(t *testing.T) {
cs := compositeStore{
stores: []compositeStoreEntry{
{model.TimeFromUnix(10), mockStoreLabelVolume{mockStore: mockStore(0), value: &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{{Name: "foo", Value: "bar", Volume: 15}}, Limit: 10,
{model.TimeFromUnix(10), mockStoreSeriesVolume{mockStore: mockStore(0), value: &logproto.VolumeResponse{
Volumes: []logproto.Volume{{Name: `{foo="bar"}`, Value: "", Volume: 15}}, Limit: 10,
}}},
{model.TimeFromUnix(20), mockStoreLabelVolume{mockStore: mockStore(1), err: errors.New("something bad")}},
{model.TimeFromUnix(20), mockStoreSeriesVolume{mockStore: mockStore(1), err: errors.New("something bad")}},
},
}
volumes, err := cs.LabelVolume(context.Background(), "fake", 10001, 20001, 10, nil)
volumes, err := cs.SeriesVolume(context.Background(), "fake", 10001, 20001, 10, nil)
require.Error(t, err, "something bad")
require.Nil(t, volumes)
})

@ -23,7 +23,7 @@ type BaseReader interface {
LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error)
LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error)
Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error)
LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error)
SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error)
}
type Reader interface {
@ -119,11 +119,11 @@ func (m monitoredReaderWriter) Stats(ctx context.Context, userID string, from, t
return sts, nil
}
func (m monitoredReaderWriter) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
var vol *logproto.LabelVolumeResponse
if err := instrument.CollectedRequest(ctx, "label_volume", instrument.NewHistogramCollector(m.metrics.indexQueryLatency), instrument.ErrorCode, func(ctx context.Context) error {
func (m monitoredReaderWriter) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
var vol *logproto.VolumeResponse
if err := instrument.CollectedRequest(ctx, "series_volume", instrument.NewHistogramCollector(m.metrics.indexQueryLatency), instrument.ErrorCode, func(ctx context.Context) error {
var err error
vol, err = m.rw.LabelVolume(ctx, userID, from, through, limit, matchers...)
vol, err = m.rw.SeriesVolume(ctx, userID, from, through, limit, matchers...)
return err
}); err != nil {
return nil, err

@ -1,102 +0,0 @@
package labelvolume
import (
"sort"
"sync"
"github.com/grafana/loki/pkg/logproto"
)
const (
MatchAny = "{}"
DefaultLimit = 100
)
// TODO(masslessparticle): Lock striping to reduce contention on this map
type Accumulator struct {
lock sync.RWMutex
volumes map[string]map[string]uint64
limit int32
}
func NewAccumulator(limit int32) *Accumulator {
return &Accumulator{
volumes: make(map[string]map[string]uint64),
limit: limit,
}
}
func (acc *Accumulator) AddVolumes(v map[string]map[string]uint64) {
acc.lock.Lock()
defer acc.lock.Unlock()
for name, values := range v {
if _, ok := acc.volumes[name]; !ok {
acc.volumes[name] = make(map[string]uint64)
}
for value, size := range values {
acc.volumes[name][value] += size
}
}
}
func (acc *Accumulator) Volumes() *logproto.LabelVolumeResponse {
acc.lock.RLock()
defer acc.lock.RUnlock()
return MapToLabelVolumeResponse(acc.volumes, int(acc.limit))
}
func Merge(responses []*logproto.LabelVolumeResponse, limit int32) *logproto.LabelVolumeResponse {
mergedVolumes := make(map[string]map[string]uint64)
for _, res := range responses {
if res == nil {
// Some stores return nil responses
continue
}
for _, v := range res.Volumes {
if _, ok := mergedVolumes[v.Name]; !ok {
mergedVolumes[v.Name] = make(map[string]uint64)
}
mergedVolumes[v.Name][v.Value] += v.GetVolume()
}
}
return MapToLabelVolumeResponse(mergedVolumes, int(limit))
}
func MapToLabelVolumeResponse(mergedVolumes map[string]map[string]uint64, limit int) *logproto.LabelVolumeResponse {
volumes := make([]logproto.LabelVolume, 0, len(mergedVolumes))
for name, v := range mergedVolumes {
for value, volume := range v {
volumes = append(volumes, logproto.LabelVolume{
Name: name,
Value: value,
Volume: volume,
})
}
}
sort.Slice(volumes, func(i, j int) bool {
if volumes[i].Volume == volumes[j].Volume {
if volumes[i].Name == volumes[j].Name {
return volumes[i].Value < volumes[j].Value
}
return volumes[i].Name < volumes[j].Name
}
return volumes[i].Volume > volumes[j].Volume
})
if limit < len(volumes) {
volumes = volumes[:limit]
}
return &logproto.LabelVolumeResponse{
Volumes: volumes,
Limit: int32(limit),
}
}

@ -0,0 +1,92 @@
package seriesvolume
import (
"sort"
"sync"
"github.com/grafana/loki/pkg/logproto"
)
const (
MatchAny = "{}"
DefaultLimit = 100
)
// TODO(masslessparticle): Lock striping to reduce contention on this map
type Accumulator struct {
lock sync.RWMutex
volumes map[string]uint64
limit int32
}
func NewAccumulator(limit int32) *Accumulator {
return &Accumulator{
volumes: make(map[string]uint64),
limit: limit,
}
}
func (acc *Accumulator) AddVolumes(volumes map[string]uint64) {
acc.lock.Lock()
defer acc.lock.Unlock()
for name, size := range volumes {
acc.volumes[name] += size
}
}
func (acc *Accumulator) Volumes() *logproto.VolumeResponse {
acc.lock.RLock()
defer acc.lock.RUnlock()
return MapToSeriesVolumeResponse(acc.volumes, int(acc.limit))
}
func Merge(responses []*logproto.VolumeResponse, limit int32) *logproto.VolumeResponse {
mergedVolumes := make(map[string]uint64)
for _, res := range responses {
if res == nil {
// Some stores return nil responses
continue
}
for _, v := range res.Volumes {
mergedVolumes[v.Name] += v.GetVolume()
}
}
return MapToSeriesVolumeResponse(mergedVolumes, int(limit))
}
func MapToSeriesVolumeResponse(mergedVolumes map[string]uint64, limit int) *logproto.VolumeResponse {
volumes := make([]logproto.Volume, 0, len(mergedVolumes))
for name, size := range mergedVolumes {
volumes = append(volumes, logproto.Volume{
Name: name,
Value: "",
Volume: size,
})
}
sort.Slice(volumes, func(i, j int) bool {
if volumes[i].Volume == volumes[j].Volume {
if volumes[i].Name == volumes[j].Name {
return volumes[i].Value < volumes[j].Value
}
return volumes[i].Name < volumes[j].Name
}
return volumes[i].Volume > volumes[j].Volume
})
if limit < len(volumes) {
volumes = volumes[:limit]
}
return &logproto.VolumeResponse{
Volumes: volumes,
Limit: int32(limit),
}
}

@ -275,19 +275,19 @@ func (s *GatewayClient) GetStats(ctx context.Context, in *logproto.IndexStatsReq
return s.grpcClient.GetStats(ctx, in, opts...)
}
func (s *GatewayClient) GetLabelVolume(ctx context.Context, in *logproto.LabelVolumeRequest, opts ...grpc.CallOption) (*logproto.LabelVolumeResponse, error) {
func (s *GatewayClient) GetSeriesVolume(ctx context.Context, in *logproto.VolumeRequest, opts ...grpc.CallOption) (*logproto.VolumeResponse, error) {
if s.cfg.Mode == indexgateway.RingMode {
var (
resp *logproto.LabelVolumeResponse
resp *logproto.VolumeResponse
err error
)
err = s.ringModeDo(ctx, func(client logproto.IndexGatewayClient) error {
resp, err = client.GetLabelVolume(ctx, in, opts...)
resp, err = client.GetSeriesVolume(ctx, in, opts...)
return err
})
return resp, err
}
return s.grpcClient.GetLabelVolume(ctx, in, opts...)
return s.grpcClient.GetSeriesVolume(ctx, in, opts...)
}
func (s *GatewayClient) doQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error {

@ -130,11 +130,12 @@ func (c *IndexGatewayClientStore) Stats(ctx context.Context, userID string, from
return resp, nil
}
func (c *IndexGatewayClientStore) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
resp, err := c.client.GetLabelVolume(ctx, &logproto.LabelVolumeRequest{
func (c *IndexGatewayClientStore) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
resp, err := c.client.GetSeriesVolume(ctx, &logproto.VolumeRequest{
From: from,
Through: through,
Matchers: (&syntax.MatchersExpr{Mts: matchers}).String(),
Limit: limit,
})
if err != nil {
if isUnimplementedCallError(err) && c.fallbackStore != nil {
@ -142,7 +143,7 @@ func (c *IndexGatewayClientStore) LabelVolume(ctx context.Context, userID string
// Note: this is likely a noop anyway since only
// tsdb+ enables this and the prior index returns an
// empty response.
return c.fallbackStore.LabelVolume(ctx, userID, from, through, limit, matchers...)
return c.fallbackStore.SeriesVolume(ctx, userID, from, through, limit, matchers...)
}
return nil, err
}

@ -717,6 +717,6 @@ func (c *indexReaderWriter) Stats(_ context.Context, _ string, _, _ model.Time,
}
// old index stores do not implement label volume -- skip
func (c *indexReaderWriter) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (c *indexReaderWriter) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return nil, nil
}

@ -6,7 +6,7 @@ import (
"sort"
"sync"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
@ -298,18 +298,18 @@ func (g *Gateway) GetStats(ctx context.Context, req *logproto.IndexStatsRequest)
return g.indexQuerier.Stats(ctx, instanceID, req.From, req.Through, matchers...)
}
func (g *Gateway) GetLabelVolume(ctx context.Context, req *logproto.LabelVolumeRequest) (*logproto.LabelVolumeResponse, error) {
func (g *Gateway) GetSeriesVolume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
instanceID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
matchers, err := syntax.ParseMatchers(req.Matchers)
if err != nil && req.Matchers != labelvolume.MatchAny {
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
return g.indexQuerier.LabelVolume(ctx, instanceID, req.From, req.Through, req.GetLimit(), matchers...)
return g.indexQuerier.SeriesVolume(ctx, instanceID, req.From, req.Through, req.GetLimit(), matchers...)
}
type failingIndexClient struct{}

@ -245,9 +245,9 @@ func TestGateway_QueryIndex_multistore(t *testing.T) {
require.Len(t, expectedQueries, 0)
}
func TestLabelVolume(t *testing.T) {
func TestSeriesVolume(t *testing.T) {
indexQuerier := newIngesterQuerierMock()
indexQuerier.On("LabelVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{
indexQuerier.On("SeriesVolume", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&logproto.VolumeResponse{Volumes: []logproto.Volume{
{Name: "bar", Value: "baz", Volume: 38},
}}, nil)
@ -255,10 +255,10 @@ func TestLabelVolume(t *testing.T) {
require.NoError(t, err)
ctx := user.InjectOrgID(context.Background(), "test")
vol, err := gateway.GetLabelVolume(ctx, &logproto.LabelVolumeRequest{Matchers: "{}"})
vol, err := gateway.GetSeriesVolume(ctx, &logproto.VolumeRequest{Matchers: "{}"})
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{Volumes: []logproto.LabelVolume{
require.Equal(t, &logproto.VolumeResponse{Volumes: []logproto.Volume{
{Name: "bar", Value: "baz", Volume: 38},
}}, vol)
}
@ -272,12 +272,12 @@ func newIngesterQuerierMock() *indexQuerierMock {
return &indexQuerierMock{}
}
func (i *indexQuerierMock) LabelVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (i *indexQuerierMock) SeriesVolume(_ context.Context, userID string, from, through model.Time, _ int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
args := i.Called(userID, from, through, matchers)
if args.Get(0) == nil {
return nil, args.Error(1)
}
return args.Get(0).(*logproto.LabelVolumeResponse), args.Error(1)
return args.Get(0).(*logproto.VolumeResponse), args.Error(1)
}

@ -783,12 +783,12 @@ func (t *tenantHeads) Stats(ctx context.Context, userID string, from, through mo
return idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}
func (t *tenantHeads) LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
func (t *tenantHeads) SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
idx, ok := t.tenantIndex(userID, from, through)
if !ok {
return nil
}
return idx.LabelVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
return idx.SeriesVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}
// helper only used in building TSDBs

@ -53,7 +53,7 @@ type Index interface {
LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error)
LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error)
Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error
LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error
SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error
}
type NoopIndex struct{}
@ -81,6 +81,6 @@ func (NoopIndex) Stats(_ context.Context, _ string, _, _ model.Time, _ IndexStat
func (NoopIndex) SetChunkFilterer(_ chunk.RequestChunkFilterer) {}
func (NoopIndex) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ LabelVolumeAccumulator, _ *index.ShardAnnotation, _ shouldIncludeChunk, _ ...*labels.Matcher) error {
func (NoopIndex) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ SeriesVolumeAccumulator, _ *index.ShardAnnotation, _ shouldIncludeChunk, _ ...*labels.Matcher) error {
return nil
}

@ -4,7 +4,7 @@ import (
"context"
"time"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
@ -48,9 +48,9 @@ type IndexStatsAccumulator interface {
Stats() stats.Stats
}
type LabelVolumeAccumulator interface {
AddVolumes(map[string]map[string]uint64)
Volumes() *logproto.LabelVolumeResponse
type SeriesVolumeAccumulator interface {
AddVolumes(map[string]uint64)
Volumes() *logproto.VolumeResponse
}
func NewIndexClient(idx Index, opts IndexClientOptions) *IndexClient {
@ -243,7 +243,10 @@ func (c *IndexClient) Stats(ctx context.Context, userID string, from, through mo
return &res, nil
}
func (c *IndexClient) LabelVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (c *IndexClient) SeriesVolume(ctx context.Context, userID string, from, through model.Time, limit int32, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "IndexClient.SeriesVolume")
defer sp.Finish()
matchers, shard, err := cleanMatchers(matchers...)
if err != nil {
return nil, err
@ -258,13 +261,22 @@ func (c *IndexClient) LabelVolume(ctx context.Context, userID string, from, thro
})
})
acc := labelvolume.NewAccumulator(limit)
acc := seriesvolume.NewAccumulator(limit)
for _, interval := range intervals {
if err := c.idx.LabelVolume(ctx, userID, interval.Start, interval.End, acc, shard, nil, matchers...); err != nil {
if err := c.idx.SeriesVolume(ctx, userID, interval.Start, interval.End, acc, shard, nil, matchers...); err != nil {
return nil, err
}
}
sp.LogKV(
"from", from.Time(),
"through", through.Time(),
"matchers", syntax.MatchersString(matchers),
"shard", shard,
"intervals", len(intervals),
"limit", limit,
)
if err != nil {
return nil, err
}

@ -217,7 +217,7 @@ func TestIndexClient_Stats(t *testing.T) {
}
}
func TestIndexClient_LabelVolume(t *testing.T) {
func TestIndexClient_SeriesVolume(t *testing.T) {
tempDir := t.TempDir()
tableRange := config.TableRange{
Start: 0,
@ -272,26 +272,27 @@ func TestIndexClient_LabelVolume(t *testing.T) {
indexClient := NewIndexClient(idx, IndexClientOptions{UseBloomFilters: true})
t.Run("it returns label volumes from the whole index", func(t *testing.T) {
vol, err := indexClient.LabelVolume(context.Background(), "", indexStartYesterday, indexStartToday+1000, 10, nil...)
t.Run("it returns series volumes from the whole index", func(t *testing.T) {
vol, err := indexClient.SeriesVolume(context.Background(), "", indexStartYesterday, indexStartToday+1000, 10, nil...)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 300 * 1024},
{Name: "fizz", Value: "buzz", Volume: 200 * 1024},
{Name: "ping", Value: "pong", Volume: 100 * 1024},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 200 * 1024},
{Name: `{fizz="buzz", foo="bar"}`, Value: "", Volume: 100 * 1024},
{Name: `{fizz="buzz"}`, Value: "", Volume: 100 * 1024},
{Name: `{ping="pong"}`, Value: "", Volume: 100 * 1024},
},
Limit: 10}, vol)
})
t.Run("it returns largest label from the index", func(t *testing.T) {
vol, err := indexClient.LabelVolume(context.Background(), "", indexStartYesterday, indexStartToday+1000, 1, nil...)
t.Run("it returns largest series from the index", func(t *testing.T) {
vol, err := indexClient.SeriesVolume(context.Background(), "", indexStartYesterday, indexStartToday+1000, 1, nil...)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: 300 * 1024},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: 200 * 1024},
},
Limit: 1}, vol)
})

@ -125,13 +125,13 @@ func (i *indexShipperQuerier) Stats(ctx context.Context, userID string, from, th
return idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}
func (i *indexShipperQuerier) LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
func (i *indexShipperQuerier) SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
idx, err := i.indices(ctx, from, through, userID)
if err != nil {
return err
}
return idx.LabelVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
return idx.SeriesVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}
type resultAccumulator struct {

@ -73,10 +73,10 @@ func (f LazyIndex) Stats(ctx context.Context, userID string, from, through model
return i.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}
func (f LazyIndex) LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
func (f LazyIndex) SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
i, err := f()
if err != nil {
return err
}
return i.LabelVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
return i.SeriesVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
}

@ -337,8 +337,8 @@ func (i *MultiIndex) Stats(ctx context.Context, userID string, from, through mod
})
}
func (i *MultiIndex) LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
func (i *MultiIndex) SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error {
return idx.LabelVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
return idx.SeriesVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...)
})
}

@ -93,6 +93,6 @@ func (m *MultiTenantIndex) Stats(ctx context.Context, userID string, from, throu
return m.idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, withTenantLabelMatcher(userID, matchers)...)
}
func (m *MultiTenantIndex) LabelVolume(ctx context.Context, userID string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
return m.idx.LabelVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, withTenantLabelMatcher(userID, matchers)...)
func (m *MultiTenantIndex) SeriesVolume(ctx context.Context, userID string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error {
return m.idx.SeriesVolume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, withTenantLabelMatcher(userID, matchers)...)
}

@ -8,6 +8,8 @@ import (
"path/filepath"
"time"
"github.com/opentracing/opentracing-go"
"github.com/go-kit/log/level"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
@ -309,9 +311,42 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim
})
}
func (i *TSDBIndex) LabelVolume(ctx context.Context, _ string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error {
volumes := make(map[string]map[string]uint64)
// SeriesVolume returns the volumes of the series described by the passed
// matchers by the names of the passed matchers. All non-requested labels are
// aggregated into the requested series.
//
// ex: Imagine we have two labels: 'foo' and 'fizz' each with two values 'a'
// and 'b'. Called with the matcher `{foo="a"}`, SeriesVolume returns the
// aggregated size of the series `{foo="a"}`. If SeriesVolume with
// `{foo=~".+", fizz=~".+"}, it returns the series volumes aggregated as follows:
//
// {foo="a", fizz="a"}
// {foo="a", fizz="b"}
// {foo="b", fizz="a"}
// {foo="b", fizz="b"}
func (i *TSDBIndex) SeriesVolume(ctx context.Context, _ string, from, through model.Time, acc SeriesVolumeAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error {
sp, ctx := opentracing.StartSpanFromContext(ctx, "Index.SeriesVolume")
defer sp.Finish()
var matchAll bool
labelsToMatch := make(map[string]struct{})
for _, m := range matchers {
if m.Name == "" {
matchAll = true
continue
}
if m.Name == TenantLabel {
continue
}
labelsToMatch[m.Name] = struct{}{}
}
seriesNames := make(map[uint64]string)
seriesLabels := labels.Labels(make([]labels.Label, 0, len(labelsToMatch)))
volumes := make(map[string]uint64)
err := i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error {
var ls labels.Labels
var filterer chunk.Filterer
@ -335,12 +370,21 @@ func (i *TSDBIndex) LabelVolume(ctx context.Context, _ string, from, through mod
}
if stats.Entries > 0 {
for _, l := range []labels.Label(ls) {
if _, ok := volumes[l.Name]; !ok {
volumes[l.Name] = make(map[string]uint64)
seriesLabels = seriesLabels[:0]
for _, l := range ls {
if _, ok := labelsToMatch[l.Name]; l.Name != TenantLabel && matchAll || ok {
seriesLabels = append(seriesLabels, l)
}
volumes[l.Name][l.Value] += stats.KB << 10 // Return bytes
}
// If the labels are < 1k, this does not alloc
// https://github.com/prometheus/prometheus/pull/8025
hash := seriesLabels.Hash()
if _, ok := seriesNames[hash]; !ok {
seriesNames[hash] = seriesLabels.String()
}
volumes[seriesNames[hash]] += stats.KB << 10 // Return bytes
}
}
return p.Err()

@ -9,7 +9,7 @@ import (
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/stores/index/labelvolume"
"github.com/grafana/loki/pkg/storage/stores/index/seriesvolume"
"github.com/grafana/loki/pkg/storage/stores/index/stats"
"github.com/go-kit/log"
@ -370,10 +370,11 @@ func TestTSDBIndex_Stats(t *testing.T) {
}
}
func TestTSDBIndex_LabelVolume(t *testing.T) {
func TestTSDBIndex_SeriesVolume(t *testing.T) {
series := []LoadableSeries{
{
Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`),
Labels: mustParseLabels(`{foo="bar", fizz="buzz", __loki_tenant__="fake"}`),
Chunks: []index.ChunkMeta{
{
MinTime: 0,
@ -392,7 +393,7 @@ func TestTSDBIndex_LabelVolume(t *testing.T) {
},
},
{
Labels: mustParseLabels(`{foo="bar", ping="pong"}`),
Labels: mustParseLabels(`{foo="bar", fizz="fizz", __loki_tenant__="fake"}`),
Chunks: []index.ChunkMeta{
{
MinTime: 0,
@ -416,39 +417,69 @@ func TestTSDBIndex_LabelVolume(t *testing.T) {
tempDir := t.TempDir()
tsdbIndex := BuildIndex(t, tempDir, series)
t.Run("it matches all the series", func(t *testing.T) {
t.Run("it matches all the series when the match all matcher is passed", func(t *testing.T) {
matcher := labels.MustNewMatcher(labels.MatchEqual, "", "")
acc := labelvolume.NewAccumulator(10)
err := tsdbIndex.LabelVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: (10 + 20 + 30 + 40) * 1024},
{Name: "ping", Value: "pong", Volume: (30 + 40) * 1024},
{Name: "fizz", Value: "buzz", Volume: (10 + 20) * 1024},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{fizz="fizz", foo="bar"}`, Value: "", Volume: (30 + 40) * 1024},
{Name: `{fizz="buzz", foo="bar"}`, Volume: (10 + 20) * 1024},
},
Limit: 10}, acc.Volumes())
})
t.Run("it ignores the tenant label matcher", func(t *testing.T) {
matcher := []*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "fizz", ".+"),
labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"),
}
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, withTenantLabelMatcher("fake", matcher)...)
require.NoError(t, err)
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{fizz="fizz", foo="bar"}`, Value: "", Volume: (30 + 40) * 1024},
{Name: `{fizz="buzz", foo="bar"}`, Volume: (10 + 20) * 1024},
},
Limit: 10}, acc.Volumes())
})
t.Run("it matches none of the series", func(t *testing.T) {
matcher := labels.MustNewMatcher(labels.MatchEqual, "foo", "baz")
acc := labelvolume.NewAccumulator(10)
err := tsdbIndex.LabelVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{},
Limit: 10}, acc.Volumes())
})
t.Run("it matches a subset", func(t *testing.T) {
matcher := labels.MustNewMatcher(labels.MatchEqual, "ping", "pong")
acc := labelvolume.NewAccumulator(10)
err := tsdbIndex.LabelVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
t.Run("it only returns results for the labels in the matcher", func(t *testing.T) {
matcher := labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
require.NoError(t, err)
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{foo="bar"}`, Value: "", Volume: (10 + 20 + 30 + 40) * 1024},
},
Limit: 10}, acc.Volumes())
})
t.Run("it returns results for label names in matchers", func(t *testing.T) {
matchers := []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"),
labels.MustNewMatcher(labels.MatchRegexp, "fizz", ".+"),
}
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matchers...)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{
{Name: "foo", Value: "bar", Volume: (30 + 40) * 1024},
{Name: "ping", Value: "pong", Volume: (30 + 40) * 1024},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{
{Name: `{fizz="fizz", foo="bar"}`, Value: "", Volume: (30 + 40) * 1024},
{Name: `{fizz="buzz", foo="bar"}`, Volume: (10 + 20) * 1024},
},
Limit: 10}, acc.Volumes())
})
@ -458,12 +489,12 @@ func TestTSDBIndex_LabelVolume(t *testing.T) {
defer tsdbIndex.SetChunkFilterer(nil)
matcher := labels.MustNewMatcher(labels.MatchEqual, "", "")
acc := labelvolume.NewAccumulator(10)
err := tsdbIndex.LabelVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
acc := seriesvolume.NewAccumulator(10)
err := tsdbIndex.SeriesVolume(context.Background(), "fake", 0, 20, acc, nil, nil, matcher)
require.NoError(t, err)
require.Equal(t, &logproto.LabelVolumeResponse{
Volumes: []logproto.LabelVolume{},
require.Equal(t, &logproto.VolumeResponse{
Volumes: []logproto.Volume{},
Limit: 10}, acc.Volumes())
})
}

@ -259,7 +259,7 @@ func (m *mockChunkStore) Stats(_ context.Context, _ string, _, _ model.Time, _ .
return nil, nil
}
func (m *mockChunkStore) LabelVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.LabelVolumeResponse, error) {
func (m *mockChunkStore) SeriesVolume(_ context.Context, _ string, _, _ model.Time, _ int32, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) {
return nil, nil
}

@ -101,7 +101,7 @@ func WriteIndexStatsResponseJSON(r *stats.Stats, w io.Writer) error {
// WriteIndexStatsResponseJSON marshals a gatewaypb.Stats to JSON and then
// writes it to the provided io.Writer.
func WriteLabelVolumeResponseJSON(r *logproto.LabelVolumeResponse, w io.Writer) error {
func WriteSeriesVolumeResponseJSON(r *logproto.VolumeResponse, w io.Writer) error {
s := jsoniter.ConfigFastest.BorrowStream(w)
defer jsoniter.ConfigFastest.ReturnStream(s)
s.WriteVal(r)

Loading…
Cancel
Save