diff --git a/pkg/api/api.go b/pkg/api/api.go index 6171150ae8..6f731e6e8d 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -20,6 +20,7 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager" "github.com/cortexproject/cortex/pkg/alertmanager/alertmanagerpb" + "github.com/cortexproject/cortex/pkg/chunksgateway" "github.com/cortexproject/cortex/pkg/compactor" "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/distributor" @@ -367,6 +368,11 @@ func (a *API) RegisterStoreGateway(s *storegateway.StoreGateway) { a.RegisterRoute("/store-gateway/ring", http.HandlerFunc(s.RingHandler), false, "GET", "POST") } +// RegisterStoreGateway registers the ring UI page associated with the store-gateway. +func (a *API) RegisterChunksGateway(s *chunksgateway.ChunksGateway) { + storegatewaypb.RegisterChunksGatewayServer(a.server.GRPC, s) +} + // RegisterCompactor registers the ring UI page associated with the compactor. func (a *API) RegisterCompactor(c *compactor.Compactor) { a.indexPage.AddLink(SectionAdminEndpoints, "/compactor/ring", "Compactor Ring Status") diff --git a/pkg/chunksgateway/gateway.go b/pkg/chunksgateway/gateway.go new file mode 100644 index 0000000000..626f0c6b29 --- /dev/null +++ b/pkg/chunksgateway/gateway.go @@ -0,0 +1,120 @@ +package chunksgateway + +import ( + "context" + "time" + + cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/storegateway" + "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/services" + "github.com/cortexproject/cortex/pkg/util/validation" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/extprom" + "github.com/weaveworks/common/logging" + + "github.com/cortexproject/cortex/pkg/storage/bucket" +) + +const ( + syncReasonInitial = "initial" + syncReasonPeriodic = "periodic" +) + +type ChunksGateway struct { + services.Service + + logger log.Logger + + gatewayCfg storegateway.Config + storageCfg cortex_tsdb.BlocksStorageConfig + + stores *storegateway.BucketStores +} + +func NewChunksGateway(gatewayCfg storegateway.Config, storageCfg cortex_tsdb.BlocksStorageConfig, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*ChunksGateway, error) { + bucketClient, err := createBucketClient(storageCfg, logger, reg) + if err != nil { + return nil, err + } + + return newChunksGateway(gatewayCfg, storageCfg, bucketClient, limits, logLevel, logger, reg) +} + +func newChunksGateway(gatewayCfg storegateway.Config, storageCfg cortex_tsdb.BlocksStorageConfig, bucketClient objstore.Bucket, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*ChunksGateway, error) { + var err error + + g := &ChunksGateway{ + gatewayCfg: gatewayCfg, + storageCfg: storageCfg, + logger: logger, + } + + // Init sharding strategy. + shardingStrategy := storegateway.NewNoShardingStrategy() + + g.stores, err = storegateway.NewBucketStores(storageCfg, shardingStrategy, bucketClient, false, limits, logLevel, logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "store-gateway"}, reg)) + if err != nil { + return nil, errors.Wrap(err, "create bucket stores") + } + + g.Service = services.NewBasicService(g.starting, g.running, g.stopping) + + return g, nil +} + +func (g *ChunksGateway) starting(ctx context.Context) (err error) { + if err = g.stores.InitialSync(ctx); err != nil { + return errors.Wrap(err, "initial blocks synchronization") + } + + return nil +} + +func (g *ChunksGateway) running(ctx context.Context) error { + // Apply a jitter to the sync frequency in order to increase the probability + // of hitting the shared cache (if any). + syncTicker := time.NewTicker(util.DurationWithJitter(g.storageCfg.BucketStore.SyncInterval, 0.2)) + defer syncTicker.Stop() + + for { + select { + case <-syncTicker.C: + g.syncStores(ctx, syncReasonPeriodic) + case <-ctx.Done(): + return nil + } + } +} + +func (g *ChunksGateway) stopping(error) error { + return nil +} + +func (g *ChunksGateway) syncStores(ctx context.Context, reason string) { + level.Info(g.logger).Log("msg", "synchronizing TSDB blocks for all users", "reason", reason) + + if err := g.stores.SyncBlocks(ctx); err != nil { + level.Warn(g.logger).Log("msg", "failed to synchronize TSDB blocks", "reason", reason, "err", err) + } else { + level.Info(g.logger).Log("msg", "successfully synchronized TSDB blocks for all users", "reason", reason) + } +} + +func createBucketClient(cfg cortex_tsdb.BlocksStorageConfig, logger log.Logger, reg prometheus.Registerer) (objstore.Bucket, error) { + bucketClient, err := bucket.NewClient(context.Background(), cfg.Bucket, "store-gateway", logger, reg) + if err != nil { + return nil, errors.Wrap(err, "create bucket client") + } + + return bucketClient, nil +} + +func (g *ChunksGateway) Chunks(srv storegatewaypb.ChunksGateway_ChunksServer) (err error) { + return g.stores.Chunks(srv) +} diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 83496d3565..314d8e2518 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -21,6 +21,7 @@ import ( "google.golang.org/grpc/health/grpc_health_v1" "gopkg.in/yaml.v2" + "github.com/cortexproject/cortex/pkg/chunksgateway" "github.com/cortexproject/cortex/pkg/util/grpcclient" "github.com/cortexproject/cortex/pkg/alertmanager" @@ -308,14 +309,15 @@ type Cortex struct { QuerierEngine v1.QueryEngine QueryFrontendTripperware tripperware.Tripperware - Ruler *ruler.Ruler - RulerStorage rulestore.RuleStore - ConfigAPI *configAPI.API - ConfigDB db.DB - Alertmanager *alertmanager.MultitenantAlertmanager - Compactor *compactor.Compactor - StoreGateway *storegateway.StoreGateway - MemberlistKV *memberlist.KVInitService + Ruler *ruler.Ruler + RulerStorage rulestore.RuleStore + ConfigAPI *configAPI.API + ConfigDB db.DB + Alertmanager *alertmanager.MultitenantAlertmanager + Compactor *compactor.Compactor + StoreGateway *storegateway.StoreGateway + ChunksGateway *chunksgateway.ChunksGateway + MemberlistKV *memberlist.KVInitService // Queryables that the querier should use to query the long // term storage. It depends on the storage engine used. diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 64cce59870..73e43f4b88 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -27,6 +27,7 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager" "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" "github.com/cortexproject/cortex/pkg/api" + "github.com/cortexproject/cortex/pkg/chunksgateway" "github.com/cortexproject/cortex/pkg/compactor" configAPI "github.com/cortexproject/cortex/pkg/configs/api" "github.com/cortexproject/cortex/pkg/configs/db" @@ -81,6 +82,7 @@ const ( AlertManager string = "alertmanager" Compactor string = "compactor" StoreGateway string = "store-gateway" + ChunksGateway string = "chunks-gateway" MemberlistKV string = "memberlist-kv" TenantDeletion string = "tenant-deletion" Purger string = "purger" @@ -681,6 +683,20 @@ func (t *Cortex) initStoreGateway() (serv services.Service, err error) { return t.StoreGateway, nil } +func (t *Cortex) initChunksGateway() (serv services.Service, err error) { + t.Cfg.StoreGateway.ShardingRing.ListenPort = t.Cfg.Server.GRPCListenPort + + t.ChunksGateway, err = chunksgateway.NewChunksGateway(t.Cfg.StoreGateway, t.Cfg.BlocksStorage, t.Overrides, t.Cfg.Server.LogLevel, util_log.Logger, prometheus.DefaultRegisterer) + if err != nil { + return nil, err + } + + // Expose HTTP endpoints. + t.API.RegisterChunksGateway(t.ChunksGateway) + + return t.ChunksGateway, nil +} + func (t *Cortex) initMemberlistKV() (services.Service, error) { reg := prometheus.DefaultRegisterer t.Cfg.MemberlistKV.MetricsRegisterer = reg @@ -759,6 +775,7 @@ func (t *Cortex) setupModuleManager() error { mm.RegisterModule(AlertManager, t.initAlertManager) mm.RegisterModule(Compactor, t.initCompactor) mm.RegisterModule(StoreGateway, t.initStoreGateway) + mm.RegisterModule(ChunksGateway, t.initChunksGateway) mm.RegisterModule(TenantDeletion, t.initTenantDeletionAPI, modules.UserInvisibleModule) mm.RegisterModule(Purger, nil) mm.RegisterModule(QueryScheduler, t.initQueryScheduler) @@ -790,10 +807,11 @@ func (t *Cortex) setupModuleManager() error { AlertManager: {API, MemberlistKV, Overrides}, Compactor: {API, MemberlistKV, Overrides}, StoreGateway: {API, Overrides, MemberlistKV}, + ChunksGateway: {API, Overrides, MemberlistKV}, TenantDeletion: {API, Overrides, DeleteRequestsStore}, Purger: {TenantDeletion}, TenantFederation: {Queryable}, - All: {QueryFrontend, Querier, Ingester, Distributor, Purger, StoreGateway, Ruler}, + All: {QueryFrontend, Querier, Ingester, Distributor, Purger, ChunksGateway, StoreGateway, Ruler}, } if t.Cfg.ExternalPusher != nil && t.Cfg.ExternalQueryable != nil { deps[Ruler] = []string{Overrides, DeleteRequestsStore, RulerStorage} diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 96d8df52f7..90403c0e18 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -13,6 +13,7 @@ import ( "sync" "time" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/status" @@ -33,7 +34,6 @@ import ( "github.com/thanos-io/objstore" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/shipper" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/httpgrpc" "go.uber.org/atomic" "golang.org/x/sync/errgroup" diff --git a/pkg/querier/block.go b/pkg/querier/block.go index 936879dac9..a1593bf123 100644 --- a/pkg/querier/block.go +++ b/pkg/querier/block.go @@ -4,34 +4,34 @@ import ( "math" "sort" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/cortexproject/cortex/pkg/querier/iterators" "github.com/cortexproject/cortex/pkg/querier/series" ) -func convertMatchersToLabelMatcher(matchers []*labels.Matcher) []storepb.LabelMatcher { - var converted []storepb.LabelMatcher +func convertMatchersToLabelMatcher(matchers []*labels.Matcher) []typespb.LabelMatcher { + var converted []typespb.LabelMatcher for _, m := range matchers { - var t storepb.LabelMatcher_Type + var t typespb.LabelMatcher_Type switch m.Type { case labels.MatchEqual: - t = storepb.LabelMatcher_EQ + t = typespb.EQ case labels.MatchNotEqual: - t = storepb.LabelMatcher_NEQ + t = typespb.NEQ case labels.MatchRegexp: - t = storepb.LabelMatcher_RE + t = typespb.RE case labels.MatchNotRegexp: - t = storepb.LabelMatcher_NRE + t = typespb.NRE } - converted = append(converted, storepb.LabelMatcher{ + converted = append(converted, typespb.LabelMatcher{ Type: t, Name: m.Name, Value: m.Value, @@ -42,7 +42,7 @@ func convertMatchersToLabelMatcher(matchers []*labels.Matcher) []storepb.LabelMa // Implementation of storage.SeriesSet, based on individual responses from store client. type blockQuerierSeriesSet struct { - series []*storepb.Series + series []*typespb.Series warnings annotations.Annotations // next response to process @@ -88,7 +88,7 @@ func (bqss *blockQuerierSeriesSet) Warnings() annotations.Annotations { } // newBlockQuerierSeries makes a new blockQuerierSeries. Input labels must be already sorted by name. -func newBlockQuerierSeries(lbls []labels.Label, chunks []storepb.AggrChunk) *blockQuerierSeries { +func newBlockQuerierSeries(lbls []labels.Label, chunks []typespb.AggrChunk) *blockQuerierSeries { sort.Slice(chunks, func(i, j int) bool { return chunks[i].MinTime < chunks[j].MinTime }) @@ -98,7 +98,7 @@ func newBlockQuerierSeries(lbls []labels.Label, chunks []storepb.AggrChunk) *blo type blockQuerierSeries struct { labels labels.Labels - chunks []storepb.AggrChunk + chunks []typespb.AggrChunk } func (bqs *blockQuerierSeries) Labels() labels.Labels { diff --git a/pkg/querier/block_test.go b/pkg/querier/block_test.go index 82a79b4efa..8e74d9e05d 100644 --- a/pkg/querier/block_test.go +++ b/pkg/querier/block_test.go @@ -7,6 +7,7 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" @@ -15,7 +16,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/cortexproject/cortex/pkg/util" ) @@ -28,23 +28,23 @@ func TestBlockQuerierSeries(t *testing.T) { maxTimestamp := time.Unix(10, 0) tests := map[string]struct { - series *storepb.Series + series *typespb.Series expectedMetric labels.Labels expectedSamples []model.SamplePair expectedErr string }{ "empty series": { - series: &storepb.Series{}, + series: &typespb.Series{}, expectedMetric: labels.Labels(nil), expectedErr: "no chunks", }, "should return series on success": { - series: &storepb.Series{ + series: &typespb.Series{ Labels: []labelpb.ZLabel{ {Name: "foo", Value: "bar"}, }, - Chunks: []storepb.AggrChunk{ - {MinTime: minTimestamp.Unix() * 1000, MaxTime: maxTimestamp.Unix() * 1000, Raw: &storepb.Chunk{Type: storepb.Chunk_XOR, Data: mockTSDBChunkData()}}, + Chunks: []typespb.AggrChunk{ + {MinTime: minTimestamp.Unix() * 1000, MaxTime: maxTimestamp.Unix() * 1000, Raw: &typespb.Chunk{Type: typespb.XOR, Data: mockTSDBChunkData()}}, }, }, expectedMetric: labels.Labels{ @@ -56,10 +56,10 @@ func TestBlockQuerierSeries(t *testing.T) { }, }, "should return error on failure while reading encoded chunk data": { - series: &storepb.Series{ + series: &typespb.Series{ Labels: []labelpb.ZLabel{{Name: "foo", Value: "bar"}}, - Chunks: []storepb.AggrChunk{ - {MinTime: minTimestamp.Unix() * 1000, MaxTime: maxTimestamp.Unix() * 1000, Raw: &storepb.Chunk{Type: storepb.Chunk_XOR, Data: []byte{0, 1}}}, + Chunks: []typespb.AggrChunk{ + {MinTime: minTimestamp.Unix() * 1000, MaxTime: maxTimestamp.Unix() * 1000, Raw: &typespb.Chunk{Type: typespb.XOR, Data: []byte{0, 1}}}, }, }, expectedMetric: labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, @@ -119,11 +119,11 @@ func TestBlockQuerierSeriesSet(t *testing.T) { // it as is, to also test transitions between series. bss := &blockQuerierSeriesSet{ - series: []*storepb.Series{ + series: []*typespb.Series{ // first, with one chunk. { Labels: mkZLabels("__name__", "first", "a", "a"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSineSamples(now, now.Add(100*time.Second), 3*time.Millisecond), // ceil(100 / 0.003) samples (= 33334) }, }, @@ -131,7 +131,7 @@ func TestBlockQuerierSeriesSet(t *testing.T) { // continuation of previous series. Must have exact same labels. { Labels: mkZLabels("__name__", "first", "a", "a"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSineSamples(now.Add(100*time.Second), now.Add(200*time.Second), 3*time.Millisecond), // ceil(100 / 0.003) samples more, 66668 in total }, }, @@ -139,7 +139,7 @@ func TestBlockQuerierSeriesSet(t *testing.T) { // second, with multiple chunks { Labels: mkZLabels("__name__", "second"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // unordered chunks createAggrChunkWithSineSamples(now.Add(400*time.Second), now.Add(600*time.Second), 5*time.Millisecond), // 200 / 0.005 (= 40000 samples, = 120000 in total) createAggrChunkWithSineSamples(now.Add(200*time.Second), now.Add(400*time.Second), 5*time.Millisecond), // 200 / 0.005 (= 40000 samples) @@ -150,13 +150,13 @@ func TestBlockQuerierSeriesSet(t *testing.T) { // overlapping { Labels: mkZLabels("__name__", "overlapping"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSineSamples(now, now.Add(10*time.Second), 5*time.Millisecond), // 10 / 0.005 = 2000 samples }, }, { Labels: mkZLabels("__name__", "overlapping"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // 10 / 0.005 = 2000 samples, but first 1000 are overlapping with previous series, so this chunk only contributes 1000 createAggrChunkWithSineSamples(now.Add(5*time.Second), now.Add(15*time.Second), 5*time.Millisecond), }, @@ -165,21 +165,21 @@ func TestBlockQuerierSeriesSet(t *testing.T) { // overlapping 2. Chunks here come in wrong order. { Labels: mkZLabels("__name__", "overlapping2"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // entire range overlaps with the next chunk, so this chunks contributes 0 samples (it will be sorted as second) createAggrChunkWithSineSamples(now.Add(3*time.Second), now.Add(7*time.Second), 5*time.Millisecond), }, }, { Labels: mkZLabels("__name__", "overlapping2"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // this chunk has completely overlaps previous chunk. Since its minTime is lower, it will be sorted as first. createAggrChunkWithSineSamples(now, now.Add(10*time.Second), 5*time.Millisecond), // 10 / 0.005 = 2000 samples }, }, { Labels: mkZLabels("__name__", "overlapping2"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // no samples createAggrChunkWithSineSamples(now, now, 5*time.Millisecond), }, @@ -187,7 +187,7 @@ func TestBlockQuerierSeriesSet(t *testing.T) { { Labels: mkZLabels("__name__", "overlapping2"), - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ // 2000 samples more (10 / 0.005) createAggrChunkWithSineSamples(now.Add(20*time.Second), now.Add(30*time.Second), 5*time.Millisecond), }, @@ -221,7 +221,7 @@ func verifyNextSeries(t *testing.T, ss storage.SeriesSet, labels labels.Labels, require.Equal(t, samples, count) } -func createAggrChunkWithSineSamples(minTime, maxTime time.Time, step time.Duration) storepb.AggrChunk { +func createAggrChunkWithSineSamples(minTime, maxTime time.Time, step time.Duration) typespb.AggrChunk { var samples []promql.FPoint minT := minTime.Unix() * 1000 @@ -235,11 +235,11 @@ func createAggrChunkWithSineSamples(minTime, maxTime time.Time, step time.Durati return createAggrChunk(minT, maxT, samples...) } -func createAggrChunkWithSamples(samples ...promql.FPoint) storepb.AggrChunk { +func createAggrChunkWithSamples(samples ...promql.FPoint) typespb.AggrChunk { return createAggrChunk(samples[0].T, samples[len(samples)-1].T, samples...) } -func createAggrChunk(minTime, maxTime int64, samples ...promql.FPoint) storepb.AggrChunk { +func createAggrChunk(minTime, maxTime int64, samples ...promql.FPoint) typespb.AggrChunk { // Ensure samples are sorted by timestamp. sort.Slice(samples, func(i, j int) bool { return samples[i].T < samples[j].T @@ -255,11 +255,11 @@ func createAggrChunk(minTime, maxTime int64, samples ...promql.FPoint) storepb.A appender.Append(s.T, s.F) } - return storepb.AggrChunk{ + return typespb.AggrChunk{ MinTime: minTime, MaxTime: maxTime, - Raw: &storepb.Chunk{ - Type: storepb.Chunk_XOR, + Raw: &typespb.Chunk{ + Type: typespb.XOR, Data: chunk.Bytes(), }, } @@ -295,7 +295,7 @@ func Benchmark_newBlockQuerierSeries(b *testing.B) { "label_8", "value_8", "label_9", "value_9") - chunks := []storepb.AggrChunk{ + chunks := []typespb.AggrChunk{ createAggrChunkWithSineSamples(time.Now(), time.Now().Add(-time.Hour), time.Minute), } @@ -313,17 +313,17 @@ func Benchmark_blockQuerierSeriesSet_iteration(b *testing.B) { ) // Generate series. - series := make([]*storepb.Series, 0, numSeries) + series := make([]*typespb.Series, 0, numSeries) for seriesID := 0; seriesID < numSeries; seriesID++ { lbls := mkZLabels("__name__", "test", "series_id", strconv.Itoa(seriesID)) - chunks := make([]storepb.AggrChunk, 0, numChunksPerSeries) + chunks := make([]typespb.AggrChunk, 0, numChunksPerSeries) // Create chunks with 1 sample per second. for minT := int64(0); minT < numChunksPerSeries*numSamplesPerChunk; minT += numSamplesPerChunk { chunks = append(chunks, createAggrChunkWithSineSamples(util.TimeFromMillis(minT), util.TimeFromMillis(minT+numSamplesPerChunk), time.Millisecond)) } - series = append(series, &storepb.Series{ + series = append(series, &typespb.Series{ Labels: lbls, Chunks: chunks, }) diff --git a/pkg/querier/blocks_store_balanced_set.go b/pkg/querier/blocks_store_balanced_set.go index 44b52eff8c..3d9e5d7974 100644 --- a/pkg/querier/blocks_store_balanced_set.go +++ b/pkg/querier/blocks_store_balanced_set.go @@ -60,7 +60,7 @@ func (s *blocksStoreBalancedSet) resolve(ctx context.Context) error { return nil } -func (s *blocksStoreBalancedSet) GetClientsFor(_ string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) { +func (s *blocksStoreBalancedSet) GetClientsFor(_ string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[ulid.ULID]BlocksStoreClient, error) { addresses := s.dnsProvider.Addresses() if len(addresses) == 0 { return nil, fmt.Errorf("no address resolved for the store-gateway service addresses %s", strings.Join(s.serviceAddresses, ",")) @@ -72,7 +72,7 @@ func (s *blocksStoreBalancedSet) GetClientsFor(_ string, blockIDs []ulid.ULID, e }) // Pick a non excluded client for each block. - clients := map[BlocksStoreClient][]ulid.ULID{} + clients := map[ulid.ULID]BlocksStoreClient{} for _, blockID := range blockIDs { // Pick the first non excluded store-gateway instance. @@ -86,7 +86,7 @@ func (s *blocksStoreBalancedSet) GetClientsFor(_ string, blockIDs []ulid.ULID, e return nil, errors.Wrapf(err, "failed to get store-gateway client for %s", addr) } - clients[c.(BlocksStoreClient)] = append(clients[c.(BlocksStoreClient)], blockID) + clients[blockID] = c.(BlocksStoreClient) } return clients, nil diff --git a/pkg/querier/blocks_store_balanced_set_test.go b/pkg/querier/blocks_store_balanced_set_test.go index 8f6ca606ef..1a45f7c2a8 100644 --- a/pkg/querier/blocks_store_balanced_set_test.go +++ b/pkg/querier/blocks_store_balanced_set_test.go @@ -39,7 +39,7 @@ func TestBlocksStoreBalancedSet_GetClientsFor(t *testing.T) { require.Len(t, clients, 1) var clientAddr string - for c := range clients { + for _, c := range clients { clientAddr = c.RemoteAddress() } @@ -80,14 +80,15 @@ func TestBlocksStoreBalancedSet_GetClientsFor_Exclude(t *testing.T) { serviceAddrs []string queryBlocks []ulid.ULID exclude map[ulid.ULID][]string - expectedClients map[string][]ulid.ULID + expectedClients map[ulid.ULID]string expectedErr error }{ "no exclude": { serviceAddrs: []string{"127.0.0.1"}, queryBlocks: []ulid.ULID{block1, block2}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "single instance available and excluded for a non-queried block": { @@ -96,8 +97,8 @@ func TestBlocksStoreBalancedSet_GetClientsFor_Exclude(t *testing.T) { exclude: map[ulid.ULID][]string{ block2: {"127.0.0.1"}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", }, }, "single instance available and excluded for the queried block": { @@ -115,9 +116,9 @@ func TestBlocksStoreBalancedSet_GetClientsFor_Exclude(t *testing.T) { block1: {"127.0.0.1"}, block2: {"127.0.0.2"}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block2}, - "127.0.0.2": {block1}, + expectedClients: map[ulid.ULID]string{ + block2: "127.0.0.1", + block1: "127.0.0.2", }, }, "multiple instances available and all are excluded for the queried block": { diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index 2ab1698e43..d9d894a62f 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -12,6 +12,9 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" + "github.com/cortexproject/cortex/pkg/storegateway/hintspb" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/protobuf/types" @@ -25,14 +28,11 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/pool" - "github.com/thanos-io/thanos/pkg/store/hintspb" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/strutil" "go.uber.org/atomic" "golang.org/x/sync/errgroup" grpc_metadata "google.golang.org/grpc/metadata" - "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/querier/series" "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/querysharding" @@ -45,7 +45,6 @@ import ( "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" "github.com/cortexproject/cortex/pkg/tenant" "github.com/cortexproject/cortex/pkg/util" - "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/math" "github.com/cortexproject/cortex/pkg/util/multierror" @@ -73,7 +72,17 @@ type BlocksStoreSet interface { // GetClientsFor returns the store gateway clients that should be used to // query the set of blocks in input. The exclude parameter is the map of // blocks -> store-gateway addresses that should be excluded. - GetClientsFor(userID string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, attemptedBlocksZones map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) + GetClientsFor(userID string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, attemptedBlocksZones map[ulid.ULID]map[string]int) (map[ulid.ULID]BlocksStoreClient, error) +} + +// BlocksStoreSet is the interface used to get the clients to query series on a set of blocks. +type ChunksStoreSet interface { + services.Service + + // GetClientsFor returns the store gateway clients that should be used to + // query the set of blocks in input. The exclude parameter is the map of + // blocks -> store-gateway addresses that should be excluded. + GetClientsFor(userID string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, attemptedBlocksZones map[ulid.ULID]map[string]int) (map[ulid.ULID]ChunksStoreClient, error) } // BlocksFinder is the interface used to find blocks for a given user and time range. @@ -131,6 +140,7 @@ type BlocksStoreQueryable struct { services.Service stores BlocksStoreSet + chunksStores ChunksStoreSet finder BlocksFinder consistency *BlocksConsistencyChecker logger log.Logger @@ -145,6 +155,7 @@ type BlocksStoreQueryable struct { func NewBlocksStoreQueryable( stores BlocksStoreSet, + chunkStores ChunksStoreSet, finder BlocksFinder, consistency *BlocksConsistencyChecker, limits BlocksStoreLimits, @@ -152,13 +163,14 @@ func NewBlocksStoreQueryable( logger log.Logger, reg prometheus.Registerer, ) (*BlocksStoreQueryable, error) { - manager, err := services.NewManager(stores, finder) + manager, err := services.NewManager(stores, chunkStores, finder) if err != nil { return nil, errors.Wrap(err, "register blocks storage queryable subservices") } q := &BlocksStoreQueryable{ stores: stores, + chunksStores: chunkStores, finder: finder, consistency: consistency, queryStoreAfter: queryStoreAfter, @@ -175,6 +187,7 @@ func NewBlocksStoreQueryable( } func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegateway.Config, storageCfg cortex_tsdb.BlocksStorageConfig, limits BlocksStoreLimits, logger log.Logger, reg prometheus.Registerer) (*BlocksStoreQueryable, error) { + chunksStores := newChunksStoreBalancedSet(querierCfg.GetChunksGatewayAddresses(), querierCfg.StoreGatewayClient, logger, reg) var stores BlocksStoreSet bucketClient, err := bucket.NewClient(context.Background(), storageCfg.Bucket, "querier", logger, reg) @@ -255,7 +268,7 @@ func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegatewa reg, ) - return NewBlocksStoreQueryable(stores, finder, consistency, limits, querierCfg.QueryStoreAfter, logger, reg) + return NewBlocksStoreQueryable(stores, chunksStores, finder, consistency, limits, querierCfg.QueryStoreAfter, logger, reg) } func (q *BlocksStoreQueryable) starting(ctx context.Context) error { @@ -294,6 +307,7 @@ func (q *BlocksStoreQueryable) Querier(mint, maxt int64) (storage.Querier, error maxT: maxt, finder: q.finder, stores: q.stores, + chunksStores: q.chunksStores, metrics: q.metrics, limits: q.limits, consistency: q.consistency, @@ -303,13 +317,14 @@ func (q *BlocksStoreQueryable) Querier(mint, maxt int64) (storage.Querier, error } type blocksStoreQuerier struct { - minT, maxT int64 - finder BlocksFinder - stores BlocksStoreSet - metrics *blocksStoreQueryableMetrics - consistency *BlocksConsistencyChecker - limits BlocksStoreLimits - logger log.Logger + minT, maxT int64 + finder BlocksFinder + stores BlocksStoreSet + chunksStores ChunksStoreSet + metrics *blocksStoreQueryableMetrics + consistency *BlocksConsistencyChecker + limits BlocksStoreLimits + logger log.Logger // If set, the querier manipulates the max time to not be greater than // "now - queryStoreAfter" so that most recent blocks are not queried. @@ -340,7 +355,7 @@ func (q *blocksStoreQuerier) LabelNames(ctx context.Context, matchers ...*labels convertedMatchers = convertMatchersToLabelMatcher(matchers) ) - queryFunc := func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error, error) { + queryFunc := func(clients map[ulid.ULID]BlocksStoreClient, minT, maxT int64) ([]ulid.ULID, error, error) { nameSets, warnings, queriedBlocks, err, retryableError := q.fetchLabelNamesFromStore(spanCtx, userID, clients, minT, maxT, convertedMatchers) if err != nil { return nil, err, retryableError @@ -379,7 +394,7 @@ func (q *blocksStoreQuerier) LabelValues(ctx context.Context, name string, match resultMtx sync.Mutex ) - queryFunc := func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error, error) { + queryFunc := func(clients map[ulid.ULID]BlocksStoreClient, minT, maxT int64) ([]ulid.ULID, error, error) { valueSets, warnings, queriedBlocks, err, retryableError := q.fetchLabelValuesFromStore(spanCtx, userID, name, clients, minT, maxT, matchers...) if err != nil { return nil, err, retryableError @@ -428,9 +443,10 @@ func (q *blocksStoreQuerier) selectSorted(ctx context.Context, sp *storage.Selec resultMtx sync.Mutex ) - queryFunc := func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error, error) { + queryFunc := func(clients map[ulid.ULID]BlocksStoreClient, minT, maxT int64) ([]ulid.ULID, error, error) { seriesSets, queriedBlocks, warnings, numChunks, err, retryableError := q.fetchSeriesFromStores(spanCtx, sp, userID, clients, minT, maxT, matchers, maxChunksLimit, leftChunksLimit) if err != nil { + level.Error(q.logger).Log("msg", "Error querying store-gateways", "err", err, "retryableerr", retryableError) return nil, err, retryableError } @@ -463,7 +479,7 @@ func (q *blocksStoreQuerier) selectSorted(ctx context.Context, sp *storage.Selec } func (q *blocksStoreQuerier) queryWithConsistencyCheck(ctx context.Context, logger log.Logger, minT, maxT int64, userID string, - queryFunc func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error, error)) error { + queryFunc func(clients map[ulid.ULID]BlocksStoreClient, minT, maxT int64) ([]ulid.ULID, error, error)) error { // If queryStoreAfter is enabled, we do manipulate the query maxt to query samples up until // now - queryStoreAfter, because the most recent time range is covered by ingesters. This // optimization is particularly important for the blocks storage because can be used to skip @@ -525,25 +541,23 @@ func (q *blocksStoreQuerier) queryWithConsistencyCheck(ctx context.Context, logg return err } - level.Debug(logger).Log("msg", "found store-gateway instances to query", "num instances", len(clients), "attempt", attempt) + level.Info(logger).Log("msg", "found store-gateway instances to query", "num instances", len(clients), "clients", fmt.Sprintf("%v", clients), "attempt", attempt) // Fetch series from stores. If an error occur we do not retry because retries // are only meant to cover missing blocks. queriedBlocks, err, retryableError = queryFunc(clients, minT, maxT) if err != nil { + level.Error(logger).Log("msg", "Error querying store-gateways", "err", err, "retryableerr", retryableError) return err } - level.Debug(logger).Log("msg", "received series from all store-gateways", "queried blocks", strings.Join(convertULIDsToString(queriedBlocks), " ")) + level.Info(logger).Log("msg", "received series from all store-gateways", "queried blocks", strings.Join(convertULIDsToString(queriedBlocks), " ")) resQueriedBlocks = append(resQueriedBlocks, queriedBlocks...) // Update the map of blocks we attempted to query. - for client, blockIDs := range clients { + for blockID, client := range clients { touchedStores[client.RemoteAddress()] = struct{}{} - - for _, blockID := range blockIDs { - attemptedBlocks[blockID] = append(attemptedBlocks[blockID], client.RemoteAddress()) - } + attemptedBlocks[blockID] = append(attemptedBlocks[blockID], client.RemoteAddress()) } // Ensure all expected blocks have been queried (during all tries done so far). @@ -555,7 +569,7 @@ func (q *blocksStoreQuerier) queryWithConsistencyCheck(ctx context.Context, logg return nil } - level.Debug(logger).Log("msg", "consistency check failed", "attempt", attempt, "missing blocks", strings.Join(convertULIDsToString(missingBlocks), " ")) + level.Info(logger).Log("msg", "consistency check failed", "attempt", attempt, "missing blocks", strings.Join(convertULIDsToString(missingBlocks), " ")) // The next attempt should just query the missing blocks. remainingBlocks = missingBlocks @@ -576,7 +590,7 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( ctx context.Context, sp *storage.SelectHints, userID string, - clients map[BlocksStoreClient][]ulid.ULID, + clients map[ulid.ULID]BlocksStoreClient, minT int64, maxT int64, matchers []*labels.Matcher, @@ -592,7 +606,6 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( queriedBlocks = []ulid.ULID(nil) numChunks = atomic.NewInt32(0) spanLog = spanlogger.FromContext(ctx) - queryLimiter = limiter.QueryLimiterFromContextWithFallback(ctx) reqStats = stats.FromContext(ctx) merrMtx = sync.Mutex{} merr = multierror.MultiError{} @@ -605,10 +618,12 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( convertedMatchers := convertMatchersToLabelMatcher(matchers) // Concurrently fetch series from all clients. - for c, blockIDs := range clients { + for blockID, c := range clients { // Change variables scope since it will be used in a goroutine. c := c - blockIDs := blockIDs + blockID := blockID + + level.Info(spanLog).Log("msg", "fetchSeriesFromStores", "block", blockID.String(), "client", c.RemoteAddress()) g.Go(func() error { // See: https://github.com/prometheus/prometheus/pull/8050 @@ -621,14 +636,14 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( seriesQueryStats := &hintspb.QueryStats{} skipChunks := sp != nil && sp.Func == "series" - req, err := createSeriesRequest(minT, maxT, convertedMatchers, shardingInfo, skipChunks, blockIDs) + req, err := createSelectRequest(minT, maxT, convertedMatchers, shardingInfo, skipChunks, blockID) if err != nil { return errors.Wrapf(err, "failed to create series request") } - begin := time.Now() - stream, err := c.Series(gCtx, req) + stream, err := c.Select(gCtx, req) if err != nil { + level.Warn(spanLog).Log("err", errors.Wrapf(err, "failed to fetch series from %s due to retryable error", c.RemoteAddress())) if isRetryableError(err) { level.Warn(spanLog).Log("err", errors.Wrapf(err, "failed to fetch series from %s due to retryable error", c.RemoteAddress())) merrMtx.Lock() @@ -639,7 +654,7 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( return errors.Wrapf(err, "failed to fetch series from %s", c.RemoteAddress()) } - mySeries := []*storepb.Series(nil) + mySeries := []*typespb.SelectedSeries(nil) myWarnings := annotations.Annotations(nil) myQueriedBlocks := []ulid.ULID(nil) @@ -685,12 +700,6 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( if s := resp.GetSeries(); s != nil { mySeries = append(mySeries, s) - // Add series fingerprint to query limiter; will return error if we are over the limit - limitErr := queryLimiter.AddSeries(cortexpb.FromLabelsToLabelAdapters(s.PromLabels())) - if limitErr != nil { - return validation.LimitError(limitErr.Error()) - } - // Ensure the max number of chunks limit hasn't been reached (max == 0 means disabled). if maxChunksLimit > 0 { actual := numChunks.Add(int32(len(s.Chunks))) @@ -698,17 +707,6 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( return validation.LimitError(fmt.Sprintf(errMaxChunksPerQueryLimit, util.LabelMatchersToString(matchers), maxChunksLimit)) } } - chunksSize := countChunkBytes(s) - dataSize := countDataBytes(s) - if chunkBytesLimitErr := queryLimiter.AddChunkBytes(chunksSize); chunkBytesLimitErr != nil { - return validation.LimitError(chunkBytesLimitErr.Error()) - } - if chunkLimitErr := queryLimiter.AddChunks(len(s.Chunks)); chunkLimitErr != nil { - return validation.LimitError(chunkLimitErr.Error()) - } - if dataBytesLimitErr := queryLimiter.AddDataBytes(dataSize); dataBytesLimitErr != nil { - return validation.LimitError(dataBytesLimitErr.Error()) - } } if w := resp.GetWarning(); w != "" { @@ -721,71 +719,45 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( return errors.Wrapf(err, "failed to unmarshal series hints from %s", c.RemoteAddress()) } - ids, err := convertBlockHintsToULIDs(hints.QueriedBlocks) if err != nil { return errors.Wrapf(err, "failed to parse queried block IDs from received hints") } - myQueriedBlocks = append(myQueriedBlocks, ids...) if hints.QueryStats != nil { seriesQueryStats.Merge(hints.QueryStats) } } } + myQueriedBlocks = append(myQueriedBlocks, blockID) numSeries := len(mySeries) - numSamples, chunksCount := countSamplesAndChunks(mySeries...) - chunkBytes := countChunkBytes(mySeries...) - dataBytes := countDataBytes(mySeries...) - reqStats.AddFetchedSeries(uint64(numSeries)) - reqStats.AddFetchedChunks(chunksCount) - reqStats.AddFetchedSamples(numSamples) - reqStats.AddFetchedChunkBytes(uint64(chunkBytes)) - reqStats.AddFetchedDataBytes(uint64(dataBytes)) - - level.Debug(spanLog).Log("msg", "received series from store-gateway", + level.Info(spanLog).Log("msg", "received series from index-gateway", "instance", c.RemoteAddress(), - "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), - "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) + "requested blocks", blockID.String(), + "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " "), + "found series", len(mySeries), + ) - // Use number of blocks queried to check whether we should log the query - // or not. It might be logging too much but good to understand per request - // performance. - if seriesQueryStats.BlocksQueried > 0 { - level.Info(spanLog).Log("msg", "store gateway series request stats", - "instance", c.RemoteAddress(), - "queryable_chunk_bytes_fetched", chunkBytes, - "queryable_data_bytes_fetched", dataBytes, - "blocks_queried", seriesQueryStats.BlocksQueried, - "series_merged_count", seriesQueryStats.MergedSeriesCount, - "chunks_merged_count", seriesQueryStats.MergedChunksCount, - "postings_touched", seriesQueryStats.PostingsTouched, - "postings_touched_size_sum", seriesQueryStats.PostingsTouchedSizeSum, - "postings_to_fetch", seriesQueryStats.PostingsToFetch, - "postings_fetched", seriesQueryStats.PostingsFetched, - "postings_fetch_count", seriesQueryStats.PostingsFetchCount, - "postings_fetched_size_sum", seriesQueryStats.PostingsFetchedSizeSum, - "series_touched", seriesQueryStats.SeriesTouched, - "series_touched_size_sum", seriesQueryStats.SeriesTouchedSizeSum, - "series_fetched", seriesQueryStats.SeriesFetched, - "series_fetch_count", seriesQueryStats.SeriesFetchCount, - "series_fetched_size_sum", seriesQueryStats.SeriesFetchedSizeSum, - "chunks_touched", seriesQueryStats.ChunksTouched, - "chunks_touched_size_sum", seriesQueryStats.ChunksTouchedSizeSum, - "chunks_fetched", seriesQueryStats.ChunksFetched, - "chunks_fetch_count", seriesQueryStats.ChunksFetchCount, - "chunks_fetched_size_sum", seriesQueryStats.ChunksFetchedSizeSum, - "data_downloaded_size_sum", seriesQueryStats.DataDownloadedSizeSum, - "get_all_duration", seriesQueryStats.GetAllDuration, - "merge_duration", seriesQueryStats.MergeDuration, - "response_time", time.Since(begin), - ) + chunks, err := q.fetchChunksFromStore(gCtx, userID, blockID, mySeries) + if err != nil { + return err } + level.Info(spanLog).Log("msg", "received chunks from chunks-gateway", + "instance", c.RemoteAddress(), + "requested blocks", blockID.String(), + "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " "), + "found chunks", len(chunks), + ) + + reqStats.AddFetchedSeries(uint64(numSeries)) + // Store the result. mtx.Lock() - seriesSets = append(seriesSets, &blockQuerierSeriesSet{series: mySeries}) + if len(mySeries) > 0 { + seriesSets = append(seriesSets, &blockSeriesSet{logger: q.logger, series: mySeries, chunks: chunks}) + } warnings.Merge(myWarnings) queriedBlocks = append(queriedBlocks, myQueriedBlocks...) mtx.Unlock() @@ -802,13 +774,90 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores( return seriesSets, queriedBlocks, warnings, int(numChunks.Load()), nil, merr.Err() } +func (q *blocksStoreQuerier) fetchChunksFromStore(ctx context.Context, userID string, blockID ulid.ULID, series []*typespb.SelectedSeries) ([][]*typespb.AggrChunk, error) { + chunks := make([][]*typespb.AggrChunk, len(series)) + + g, gCtx := errgroup.WithContext(ctx) + batchSize := 10000 + + for i := 0; i < len(series); i += batchSize { + j := i + batchSize + if j > len(series) { + j = len(series) + } + + start := i + end := j + + g.Go(func() error { + chks, err := q.fetchChunksForBatch(gCtx, userID, blockID, series[start:end]) + if err != nil { + return err + } + for i, chk := range chks { + chunks[start+i] = chk + } + return nil + }) + } + + if err := g.Wait(); err != nil { + return nil, err + } + + return chunks, nil +} + +func (q *blocksStoreQuerier) fetchChunksForBatch(ctx context.Context, userID string, blockID ulid.ULID, series []*typespb.SelectedSeries) ([][]*typespb.AggrChunk, error) { + clients, err := q.chunksStores.GetClientsFor(userID, []ulid.ULID{blockID}, map[ulid.ULID][]string{}, map[ulid.ULID]map[string]int{}) + if err != nil { + return nil, err + } + chunksClient := clients[blockID] + level.Info(q.logger).Log("msg", "found chunk-gateway instances to query", "num instances", len(clients), "clients", fmt.Sprintf("%v", clients)) + + chunksStream, err := chunksClient.Chunks(ctx) + if err != nil { + return nil, err + } + + chunks := make([][]*typespb.AggrChunk, 0, len(series)) + + // Get in batches of 10K series. + for _, s := range series { + refs := make([]uint64, 0, len(s.Chunks)) + for _, meta := range s.Chunks { + refs = append(refs, meta.Ref) + } + chunksStream.Send(&storepb.ChunksRequest{ + BlockId: blockID.String(), + Chunkref: refs, + }) + } + chunksStream.CloseSend() + + for { + in, err := chunksStream.Recv() + + if err == io.EOF { + break + } + if err != nil { + return nil, err + } + chunks = append(chunks, in.Chunks) + } + + return chunks, nil +} + func (q *blocksStoreQuerier) fetchLabelNamesFromStore( ctx context.Context, userID string, - clients map[BlocksStoreClient][]ulid.ULID, + clients map[ulid.ULID]BlocksStoreClient, minT int64, maxT int64, - matchers []storepb.LabelMatcher, + matchers []typespb.LabelMatcher, ) ([][]string, annotations.Annotations, []ulid.ULID, error, error) { var ( reqCtx = grpc_metadata.AppendToOutgoingContext(ctx, cortex_tsdb.TenantIDExternalLabel, userID) @@ -823,13 +872,13 @@ func (q *blocksStoreQuerier) fetchLabelNamesFromStore( ) // Concurrently fetch series from all clients. - for c, blockIDs := range clients { + for blockID, c := range clients { // Change variables scope since it will be used in a goroutine. c := c - blockIDs := blockIDs + blockID := blockID g.Go(func() error { - req, err := createLabelNamesRequest(minT, maxT, blockIDs, matchers) + req, err := createLabelNamesRequest(minT, maxT, blockID, matchers) if err != nil { return errors.Wrapf(err, "failed to create label names request") } @@ -879,7 +928,7 @@ func (q *blocksStoreQuerier) fetchLabelNamesFromStore( level.Debug(spanLog).Log("msg", "received label names from store-gateway", "instance", c, "num labels", len(namesResp.Names), - "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), + "requested blocks", blockID.String(), "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) // Store the result. @@ -907,7 +956,7 @@ func (q *blocksStoreQuerier) fetchLabelValuesFromStore( ctx context.Context, userID string, name string, - clients map[BlocksStoreClient][]ulid.ULID, + clients map[ulid.ULID]BlocksStoreClient, minT int64, maxT int64, matchers ...*labels.Matcher, @@ -925,13 +974,13 @@ func (q *blocksStoreQuerier) fetchLabelValuesFromStore( ) // Concurrently fetch series from all clients. - for c, blockIDs := range clients { + for blockID, c := range clients { // Change variables scope since it will be used in a goroutine. c := c - blockIDs := blockIDs + blockID := blockID g.Go(func() error { - req, err := createLabelValuesRequest(minT, maxT, name, blockIDs, matchers...) + req, err := createLabelValuesRequest(minT, maxT, name, blockID, matchers...) if err != nil { return errors.Wrapf(err, "failed to create label values request") } @@ -981,7 +1030,7 @@ func (q *blocksStoreQuerier) fetchLabelValuesFromStore( level.Debug(spanLog).Log("msg", "received label values from store-gateway", "instance", c.RemoteAddress(), "num values", len(valuesResp.Values), - "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), + "requested blocks", blockID.String(), "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) // Values returned need not be sorted, but we need them to be sorted so we can merge. @@ -1008,14 +1057,14 @@ func (q *blocksStoreQuerier) fetchLabelValuesFromStore( return valueSets, warnings, queriedBlocks, nil, merr.Err() } -func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, shardingInfo *storepb.ShardInfo, skipChunks bool, blockIDs []ulid.ULID) (*storepb.SeriesRequest, error) { +func createSeriesRequest(minT, maxT int64, matchers []typespb.LabelMatcher, shardingInfo *storepb.ShardInfo, skipChunks bool, blockID ulid.ULID) (*storepb.SeriesRequest, error) { // Selectively query only specific blocks. hints := &hintspb.SeriesRequestHints{ - BlockMatchers: []storepb.LabelMatcher{ + BlockMatchers: []typespb.LabelMatcher{ { - Type: storepb.LabelMatcher_RE, + Type: typespb.RE, Name: block.BlockIDLabel, - Value: strings.Join(convertULIDsToString(blockIDs), "|"), + Value: blockID.String(), }, }, EnableQueryStats: true, @@ -1030,14 +1079,43 @@ func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, shar MinTime: minT, MaxTime: maxT, Matchers: matchers, - PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, + PartialResponseStrategy: typespb.ABORT, + Hints: anyHints, + SkipChunks: skipChunks, + ShardInfo: shardingInfo, + }, nil +} + +func createSelectRequest(minT, maxT int64, matchers []typespb.LabelMatcher, shardingInfo *storepb.ShardInfo, skipChunks bool, blockID ulid.ULID) (*storepb.SelectRequest, error) { + // Selectively query only specific blocks. + hints := &hintspb.SeriesRequestHints{ + BlockMatchers: []typespb.LabelMatcher{ + { + Type: typespb.RE, + Name: block.BlockIDLabel, + Value: blockID.String(), + }, + }, + EnableQueryStats: true, + } + + anyHints, err := types.MarshalAny(hints) + if err != nil { + return nil, errors.Wrapf(err, "failed to marshal series request hints") + } + + return &storepb.SelectRequest{ + MinTime: minT, + MaxTime: maxT, + Matchers: matchers, + PartialResponseStrategy: typespb.ABORT, Hints: anyHints, SkipChunks: skipChunks, ShardInfo: shardingInfo, }, nil } -func createLabelNamesRequest(minT, maxT int64, blockIDs []ulid.ULID, matchers []storepb.LabelMatcher) (*storepb.LabelNamesRequest, error) { +func createLabelNamesRequest(minT, maxT int64, blockID ulid.ULID, matchers []typespb.LabelMatcher) (*storepb.LabelNamesRequest, error) { req := &storepb.LabelNamesRequest{ Start: minT, End: maxT, @@ -1046,11 +1124,11 @@ func createLabelNamesRequest(minT, maxT int64, blockIDs []ulid.ULID, matchers [] // Selectively query only specific blocks. hints := &hintspb.LabelNamesRequestHints{ - BlockMatchers: []storepb.LabelMatcher{ + BlockMatchers: []typespb.LabelMatcher{ { - Type: storepb.LabelMatcher_RE, + Type: typespb.RE, Name: block.BlockIDLabel, - Value: strings.Join(convertULIDsToString(blockIDs), "|"), + Value: blockID.String(), }, }, } @@ -1065,7 +1143,7 @@ func createLabelNamesRequest(minT, maxT int64, blockIDs []ulid.ULID, matchers [] return req, nil } -func createLabelValuesRequest(minT, maxT int64, label string, blockIDs []ulid.ULID, matchers ...*labels.Matcher) (*storepb.LabelValuesRequest, error) { +func createLabelValuesRequest(minT, maxT int64, label string, blockID ulid.ULID, matchers ...*labels.Matcher) (*storepb.LabelValuesRequest, error) { req := &storepb.LabelValuesRequest{ Start: minT, End: maxT, @@ -1075,11 +1153,11 @@ func createLabelValuesRequest(minT, maxT int64, label string, blockIDs []ulid.UL // Selectively query only specific blocks. hints := &hintspb.LabelValuesRequestHints{ - BlockMatchers: []storepb.LabelMatcher{ + BlockMatchers: []typespb.LabelMatcher{ { - Type: storepb.LabelMatcher_RE, + Type: typespb.RE, Name: block.BlockIDLabel, - Value: strings.Join(convertULIDsToString(blockIDs), "|"), + Value: blockID.String(), }, }, } @@ -1118,7 +1196,7 @@ func convertBlockHintsToULIDs(hints []hintspb.Block) ([]ulid.ULID, error) { } // countChunkBytes returns the size of the chunks making up the provided series in bytes -func countChunkBytes(series ...*storepb.Series) (count int) { +func countChunkBytes(series ...*typespb.Series) (count int) { for _, s := range series { for _, c := range s.Chunks { count += c.Size() @@ -1129,7 +1207,7 @@ func countChunkBytes(series ...*storepb.Series) (count int) { } // countDataBytes returns the combined size of the all series -func countDataBytes(series ...*storepb.Series) (count int) { +func countDataBytes(series ...*typespb.Series) (count int) { for _, s := range series { count += s.Size() } @@ -1138,7 +1216,7 @@ func countDataBytes(series ...*storepb.Series) (count int) { } // countSamplesAndChunks counts the number of samples and number counts from the series. -func countSamplesAndChunks(series ...*storepb.Series) (samplesCount, chunksCount uint64) { +func countSamplesAndChunks(series ...*typespb.Series) (samplesCount, chunksCount uint64) { for _, s := range series { chunksCount += uint64(len(s.Chunks)) for _, c := range s.Chunks { diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 9b05c4f94b..3c07b202a5 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -9,6 +9,8 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/go-kit/log" "github.com/gogo/protobuf/types" "github.com/oklog/ulid" @@ -28,7 +30,6 @@ import ( "github.com/thanos-io/thanos/pkg/pool" "github.com/thanos-io/thanos/pkg/store/hintspb" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/user" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -1534,18 +1535,18 @@ func TestBlocksStoreQuerier_PromQLExecution(t *testing.T) { gateway1 := &storeGatewayClientMock{remoteAddr: "1.1.1.1", mockedSeriesResponses: []*storepb.SeriesResponse{ { Result: &storepb.SeriesResponse_Series{ - Series: &storepb.Series{ + Series: &typespb.Series{ Labels: series1, - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSamples(series1Samples[:3]...), // First half. }, }, }, }, { Result: &storepb.SeriesResponse_Series{ - Series: &storepb.Series{ + Series: &typespb.Series{ Labels: series2, - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSamples(series2Samples[:3]...), }, }, @@ -1557,18 +1558,18 @@ func TestBlocksStoreQuerier_PromQLExecution(t *testing.T) { gateway2 := &storeGatewayClientMock{remoteAddr: "2.2.2.2", mockedSeriesResponses: []*storepb.SeriesResponse{ { Result: &storepb.SeriesResponse_Series{ - Series: &storepb.Series{ + Series: &typespb.Series{ Labels: series1, - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSamples(series1Samples[3:]...), // Second half. }, }, }, }, { Result: &storepb.SeriesResponse_Series{ - Series: &storepb.Series{ + Series: &typespb.Series{ Labels: series2, - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ createAggrChunkWithSamples(series2Samples[3:]...), }, }, @@ -1577,6 +1578,8 @@ func TestBlocksStoreQuerier_PromQLExecution(t *testing.T) { mockHintsResponse(block2), }} + chunkGateway := &chunkGatewayClientMock{remoteAddr: "1.1.1.1"} + stores := &blocksStoreSetMock{ Service: services.NewIdleService(nil, nil), mockedResponses: []interface{}{ @@ -1587,8 +1590,18 @@ func TestBlocksStoreQuerier_PromQLExecution(t *testing.T) { }, } + chunkStores := &chunksStoreSetMock{ + Service: services.NewIdleService(nil, nil), + mockedResponses: []interface{}{ + map[ChunksStoreClient][]ulid.ULID{ + chunkGateway: {block1}, + chunkGateway: {block2}, + }, + }, + } + // Instance the querier that will be executed to run the query. - queryable, err := NewBlocksStoreQueryable(stores, finder, NewBlocksConsistencyChecker(0, 0, logger, nil), &blocksStoreLimitsMock{}, 0, logger, nil) + queryable, err := NewBlocksStoreQueryable(stores, chunkStores, finder, NewBlocksConsistencyChecker(0, 0, logger, nil), &blocksStoreLimitsMock{}, 0, logger, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), queryable)) defer services.StopAndAwaitTerminated(context.Background(), queryable) // nolint:errcheck @@ -1621,7 +1634,32 @@ type blocksStoreSetMock struct { nextResult int } -func (m *blocksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) { +func (m *blocksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[ulid.ULID]BlocksStoreClient, error) { + if m.nextResult >= len(m.mockedResponses) { + panic("not enough mocked results") + } + + res := m.mockedResponses[m.nextResult] + m.nextResult++ + + if err, ok := res.(error); ok { + return nil, err + } + if clients, ok := res.(map[ulid.ULID]BlocksStoreClient); ok { + return clients, nil + } + + return nil, errors.New("unknown data type in the mocked result") +} + +type chunksStoreSetMock struct { + services.Service + + mockedResponses []interface{} + nextResult int +} + +func (m *chunksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[ulid.ULID]ChunksStoreClient, error) { if m.nextResult >= len(m.mockedResponses) { panic("not enough mocked results") } @@ -1632,7 +1670,7 @@ func (m *blocksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.U if err, ok := res.(error); ok { return nil, err } - if clients, ok := res.(map[BlocksStoreClient][]ulid.ULID); ok { + if clients, ok := res.(map[ulid.ULID]ChunksStoreClient); ok { return clients, nil } @@ -1649,9 +1687,40 @@ func (m *blocksFinderMock) GetBlocks(ctx context.Context, userID string, minT, m return args.Get(0).(bucketindex.Blocks), args.Get(1).(map[ulid.ULID]*bucketindex.BlockDeletionMark), args.Error(2) } +type chunkGatewayClientMock struct { + remoteAddr string + mockedChunksResponses []*storepb.ChunksResponse + mockedSeriesErr error +} + +func (m *chunkGatewayClientMock) RemoteAddress() string { + return m.remoteAddr +} + +func (m *chunkGatewayClientMock) Chunks(ctx context.Context, opts ...grpc.CallOption) (storegatewaypb.ChunksGateway_ChunksClient, error) { + chunksClient := &chunkGatewayChunksClientMock{} + + return chunksClient, m.mockedSeriesErr +} + +type chunkGatewayChunksClientMock struct { + grpc.ClientStream +} + +// Recv implements storegatewaypb.ChunksGateway_ChunksClient. +func (*chunkGatewayChunksClientMock) Recv() (*storepb.ChunksResponse, error) { + return nil, io.EOF +} + +// Send implements storegatewaypb.ChunksGateway_ChunksClient. +func (*chunkGatewayChunksClientMock) Send(*storepb.ChunksRequest) error { + return nil +} + type storeGatewayClientMock struct { remoteAddr string mockedSeriesResponses []*storepb.SeriesResponse + mockedSelectResponses []*storepb.SelectResponse mockedSeriesErr error mockedSeriesStreamErr error mockedLabelNamesResponse *storepb.LabelNamesResponse @@ -1668,6 +1737,15 @@ func (m *storeGatewayClientMock) Series(ctx context.Context, in *storepb.SeriesR return seriesClient, m.mockedSeriesErr } +func (m *storeGatewayClientMock) Select(ctx context.Context, in *storepb.SelectRequest, opts ...grpc.CallOption) (storegatewaypb.StoreGateway_SelectClient, error) { + seriesClient := &storeGatewaySelectClientMock{ + mockedResponses: m.mockedSelectResponses, + mockedSeriesStreamErr: m.mockedSeriesStreamErr, + } + + return seriesClient, m.mockedSeriesErr +} + func (m *storeGatewayClientMock) LabelNames(context.Context, *storepb.LabelNamesRequest, ...grpc.CallOption) (*storepb.LabelNamesResponse, error) { return m.mockedLabelNamesResponse, nil } @@ -1700,6 +1778,26 @@ func (m *storeGatewaySeriesClientMock) Recv() (*storepb.SeriesResponse, error) { return res, m.mockedSeriesStreamErr } +type storeGatewaySelectClientMock struct { + grpc.ClientStream + + mockedResponses []*storepb.SelectResponse + mockedSeriesStreamErr error +} + +func (m *storeGatewaySelectClientMock) Recv() (*storepb.SelectResponse, error) { + // Ensure some concurrency occurs. + time.Sleep(10 * time.Millisecond) + + if len(m.mockedResponses) == 0 { + return nil, io.EOF + } + + res := m.mockedResponses[0] + m.mockedResponses = m.mockedResponses[1:] + return res, m.mockedSeriesStreamErr +} + type blocksStoreLimitsMock struct { maxChunksPerQuery int storeGatewayTenantShardSize float64 @@ -1737,10 +1835,10 @@ func mockSeriesResponse(lbls labels.Labels, timeMillis int64, value float64) *st return &storepb.SeriesResponse{ Result: &storepb.SeriesResponse_Series{ - Series: &storepb.Series{ + Series: &typespb.Series{ Labels: labelpb.ZLabelsFromPromLabels(lbls), - Chunks: []storepb.AggrChunk{ - {MinTime: timeMillis, MaxTime: timeMillis, Raw: &storepb.Chunk{Type: storepb.Chunk_XOR, Data: chunkData}}, + Chunks: []typespb.AggrChunk{ + {MinTime: timeMillis, MaxTime: timeMillis, Raw: &typespb.Chunk{Type: typespb.XOR, Data: chunkData}}, }, }, }, @@ -1839,17 +1937,17 @@ func TestCountSamplesAndChunks(t *testing.T) { } for i, tc := range []struct { - serieses []*storepb.Series + serieses []*typespb.Series expectedChunks uint64 expectedSamples uint64 }{ { - serieses: []*storepb.Series{ + serieses: []*typespb.Series{ { - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ { - Raw: &storepb.Chunk{ - Type: storepb.Chunk_XOR, + Raw: &typespb.Chunk{ + Type: typespb.XOR, Data: c.Bytes(), }, }, @@ -1860,18 +1958,18 @@ func TestCountSamplesAndChunks(t *testing.T) { expectedChunks: 1, }, { - serieses: []*storepb.Series{ + serieses: []*typespb.Series{ { - Chunks: []storepb.AggrChunk{ + Chunks: []typespb.AggrChunk{ { - Raw: &storepb.Chunk{ - Type: storepb.Chunk_XOR, + Raw: &typespb.Chunk{ + Type: typespb.XOR, Data: c.Bytes(), }, }, { - Raw: &storepb.Chunk{ - Type: storepb.Chunk_XOR, + Raw: &typespb.Chunk{ + Type: typespb.XOR, Data: c.Bytes(), }, }, diff --git a/pkg/querier/blocks_store_replicated_set.go b/pkg/querier/blocks_store_replicated_set.go index adda8ece6d..564c0c076e 100644 --- a/pkg/querier/blocks_store_replicated_set.go +++ b/pkg/querier/blocks_store_replicated_set.go @@ -7,6 +7,7 @@ import ( "math/rand" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -36,6 +37,7 @@ type blocksStoreReplicationSet struct { shardingStrategy string balancingStrategy loadBalancingStrategy limits BlocksStoreLimits + logger log.Logger zoneAwarenessEnabled bool zoneStableShuffleSharding bool @@ -62,6 +64,7 @@ func newBlocksStoreReplicationSet( shardingStrategy: shardingStrategy, balancingStrategy: balancingStrategy, limits: limits, + logger: logger, zoneAwarenessEnabled: zoneAwarenessEnabled, zoneStableShuffleSharding: zoneStableShuffleSharding, @@ -103,7 +106,7 @@ func (s *blocksStoreReplicationSet) stopping(_ error) error { return services.StopManagerAndAwaitStopped(context.Background(), s.subservices) } -func (s *blocksStoreReplicationSet) GetClientsFor(userID string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, attemptedBlocksZones map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) { +func (s *blocksStoreReplicationSet) GetClientsFor(userID string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, attemptedBlocksZones map[ulid.ULID]map[string]int) (map[ulid.ULID]BlocksStoreClient, error) { shards := map[string][]ulid.ULID{} // If shuffle sharding is enabled, we should build a subring for the user, @@ -126,6 +129,8 @@ func (s *blocksStoreReplicationSet) GetClientsFor(userID string, blockIDs []ulid return nil, errors.Wrapf(err, "failed to get store-gateway replication set owning the block %s", blockID.String()) } + level.Info(s.logger).Log("msg", "GetClientsFor", "blockID", blockID.String(), "excluded", fmt.Sprintf("%v", exclude[blockID]), "attempted", fmt.Sprintf("%v", attemptedBlocksZones[blockID])) + // Pick a non excluded store-gateway instance. instance := getNonExcludedInstance(set, exclude[blockID], s.balancingStrategy, s.zoneAwarenessEnabled, attemptedBlocksZones[blockID]) // A valid instance should have a non-empty address. @@ -142,7 +147,7 @@ func (s *blocksStoreReplicationSet) GetClientsFor(userID string, blockIDs []ulid } } - clients := map[BlocksStoreClient][]ulid.ULID{} + clients := map[ulid.ULID]BlocksStoreClient{} // Get the client for each store-gateway. for addr, blockIDs := range shards { @@ -151,7 +156,9 @@ func (s *blocksStoreReplicationSet) GetClientsFor(userID string, blockIDs []ulid return nil, errors.Wrapf(err, "failed to get store-gateway client for %s", addr) } - clients[c.(BlocksStoreClient)] = blockIDs + for _, bid := range blockIDs { + clients[bid] = c.(BlocksStoreClient) + } } return clients, nil diff --git a/pkg/querier/blocks_store_replicated_set_test.go b/pkg/querier/blocks_store_replicated_set_test.go index 40637a07d4..3769bc7e37 100644 --- a/pkg/querier/blocks_store_replicated_set_test.go +++ b/pkg/querier/blocks_store_replicated_set_test.go @@ -72,7 +72,7 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { exclude map[ulid.ULID][]string attemptedBlocksZones map[ulid.ULID]map[string]int zoneAwarenessEnabled bool - expectedClients map[string][]ulid.ULID + expectedClients map[ulid.ULID]string expectedErr error }{ // @@ -85,8 +85,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-1", "127.0.0.1", "", []uint32{block1Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "default sharding, single instance in the ring with RF = 1 but excluded": { @@ -111,8 +112,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { exclude: map[ulid.ULID][]string{ block3: {"127.0.0.1"}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "default sharding, single instance in the ring with RF = 2": { @@ -122,8 +124,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-1", "127.0.0.1", "", []uint32{block1Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "default sharding, multiple instances in the ring with each requested block belonging to a different store-gateway and RF = 1": { @@ -136,10 +139,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block3, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1}, - "127.0.0.3": {block3}, - "127.0.0.4": {block4}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block3: "127.0.0.3", + block4: "127.0.0.4", }, }, "default sharding, multiple instances in the ring with each requested block belonging to a different store-gateway and RF = 1 but excluded": { @@ -167,10 +170,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block3, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1}, - "127.0.0.3": {block3}, - "127.0.0.4": {block4}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block3: "127.0.0.3", + block4: "127.0.0.4", }, }, "default sharding, multiple instances in the ring with multiple requested blocks belonging to the same store-gateway and RF = 2": { @@ -181,9 +184,11 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-2", "127.0.0.2", "", []uint32{block3Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2, block3, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block4}, - "127.0.0.2": {block2, block3}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.2", + block3: "127.0.0.2", + block4: "127.0.0.1", }, }, "default sharding, multiple instances in the ring with each requested block belonging to a different store-gateway and RF = 2 and some blocks excluded but with replacement available": { @@ -200,9 +205,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { block3: {"127.0.0.3"}, block1: {"127.0.0.1"}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.2": {block1}, - "127.0.0.4": {block3, block4}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.2", + block3: "127.0.0.4", + block4: "127.0.0.4", }, }, "default sharding, multiple instances in the ring are JOINING, the requested block + its replicas only belongs to JOINING instances": { @@ -215,8 +221,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.4": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.4", }, }, // @@ -230,8 +236,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-1", "127.0.0.1", "", []uint32{block1Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "shuffle sharding, single instance in the ring with RF = 1, SS = 1 but excluded": { @@ -255,8 +262,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-1", "127.0.0.1", "", []uint32{block1Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", }, }, "shuffle sharding, multiple instances in the ring with RF = 1, SS = 1": { @@ -270,8 +278,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block2, block4}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.1", + block4: "127.0.0.1", }, }, "shuffle sharding, multiple instances in the ring with RF = 1, SS = 2": { @@ -285,9 +295,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1, block4}, - "127.0.0.2": {block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.2", + block4: "127.0.0.1", }, }, "shuffle sharding, multiple instances in the ring with RF = 1, SS = 4": { @@ -301,10 +312,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { d.AddIngester("instance-4", "127.0.0.4", "", []uint32{block4Hash + 1}, ring.ACTIVE, registeredAt) }, queryBlocks: []ulid.ULID{block1, block2, block4}, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1}, - "127.0.0.2": {block2}, - "127.0.0.4": {block4}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.2", + block4: "127.0.0.4", }, }, "shuffle sharding, multiple instances in the ring with RF = 2, SS = 2 with excluded blocks but some replacement available": { @@ -322,8 +333,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { block1: {"127.0.0.1"}, block2: {"127.0.0.1"}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.2": {block1, block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.2", + block2: "127.0.0.2", }, }, "shuffle sharding, multiple instances in the ring with RF = 2, SS = 2 with excluded blocks and no replacement available": { @@ -361,9 +373,9 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { queryBlocks: []ulid.ULID{block1, block2}, zoneAwarenessEnabled: true, attemptedBlocksZones: make(map[ulid.ULID]map[string]int, 0), - expectedClients: map[string][]ulid.ULID{ - "127.0.0.1": {block1}, - "127.0.0.6": {block2}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.1", + block2: "127.0.0.6", }, }, "shuffle sharding, multiple instances in the ring with RF = 3, SS = 3, exclude and zone awareness enabled": { @@ -389,8 +401,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.6": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.6", }, }, "shuffle sharding, multiple instances in the ring with RF = 3, SS = 3, exclude 2 blocks and zone awareness enabled": { @@ -416,8 +428,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1, "3": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.8": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.8", }, }, "shuffle sharding, multiple instances in the ring with RF = 3, SS = 3, exclude 3 blocks and zone awareness enabled": { @@ -468,8 +480,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1, "2": 1, "3": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.2": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.2", }, }, "shuffle sharding, multiple instances in the ring with RF = 6, SS = 6, exclude 2 blocks and zone awareness enabled": { @@ -495,8 +507,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1, "3": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.2": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.2", }, }, // This should never happen, just to test the attemptedZoneMap works correctly. @@ -520,8 +532,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1, "3": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.2": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.2", }, }, // This should never happen, just to test the attemptedZoneMap works correctly. @@ -548,8 +560,8 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { attemptedBlocksZones: map[ulid.ULID]map[string]int{ block1: {"1": 1, "3": 1}, }, - expectedClients: map[string][]ulid.ULID{ - "127.0.0.8": {block1}, + expectedClients: map[ulid.ULID]string{ + block1: "127.0.0.8", }, }, } @@ -600,13 +612,18 @@ func TestBlocksStoreReplicationSet_GetClientsFor(t *testing.T) { assert.Equal(t, testData.expectedErr, err) if testData.expectedErr == nil { + + dedupedClients := map[string]struct{}{} + for _, c := range testData.expectedClients { + dedupedClients[c] = struct{}{} + } assert.Equal(t, testData.expectedClients, getStoreGatewayClientAddrs(clients)) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(fmt.Sprintf(` # HELP cortex_storegateway_clients The current number of store-gateway clients in the pool. # TYPE cortex_storegateway_clients gauge cortex_storegateway_clients{client="querier"} %d - `, len(testData.expectedClients))), "cortex_storegateway_clients")) + `, len(dedupedClients))), "cortex_storegateway_clients")) } }) } @@ -667,7 +684,7 @@ func TestBlocksStoreReplicationSet_GetClientsFor_ShouldSupportRandomLoadBalancin require.NoError(t, err) require.Len(t, clients, 1) - for addr := range getStoreGatewayClientAddrs(clients) { + for _, addr := range getStoreGatewayClientAddrs(clients) { distribution[addr]++ } } @@ -741,7 +758,7 @@ func TestBlocksStoreReplicationSet_GetClientsFor_ZoneAwareness(t *testing.T) { clients, err := s.GetClientsFor(userID, []ulid.ULID{block1}, nil, attemptedBlocksZone) require.NoError(t, err) require.Len(t, clients, 1) - for c := range clients { + for _, c := range clients { addr := c.RemoteAddress() parts := strings.Split(addr, ".") require.True(t, len(parts) > 3) @@ -752,10 +769,10 @@ func TestBlocksStoreReplicationSet_GetClientsFor_ZoneAwareness(t *testing.T) { } } -func getStoreGatewayClientAddrs(clients map[BlocksStoreClient][]ulid.ULID) map[string][]ulid.ULID { - addrs := map[string][]ulid.ULID{} - for c, blockIDs := range clients { - addrs[c.RemoteAddress()] = blockIDs +func getStoreGatewayClientAddrs(clients map[ulid.ULID]BlocksStoreClient) map[ulid.ULID]string { + addrs := map[ulid.ULID]string{} + for blockId, c := range clients { + addrs[blockId] = c.RemoteAddress() } return addrs } diff --git a/pkg/querier/chunks_store_balanced_set.go b/pkg/querier/chunks_store_balanced_set.go new file mode 100644 index 0000000000..109dfb901f --- /dev/null +++ b/pkg/querier/chunks_store_balanced_set.go @@ -0,0 +1,105 @@ +package querier + +import ( + "context" + "fmt" + "math/rand" + "strings" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/thanos/pkg/discovery/dns" + "github.com/thanos-io/thanos/pkg/extprom" + + "github.com/cortexproject/cortex/pkg/ring/client" + "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" + "github.com/cortexproject/cortex/pkg/util/services" +) + +// BlocksStoreClient is the interface that should be implemented by any client used +// to query a backend store-gateway. +type ChunksStoreClient interface { + storegatewaypb.ChunksGatewayClient + + // RemoteAddress returns the address of the remote store-gateway and is used to uniquely + // identify a store-gateway backend instance. + RemoteAddress() string +} + +// BlocksStoreSet implementation used when the blocks are not sharded in the store-gateway +// and so requests are balanced across the set of store-gateway instances. +type chunksStoreBalancedSet struct { + services.Service + + serviceAddresses []string + clientsPool *client.Pool + dnsProvider *dns.Provider + + logger log.Logger +} + +func newChunksStoreBalancedSet(serviceAddresses []string, clientConfig ClientConfig, logger log.Logger, reg prometheus.Registerer) *chunksStoreBalancedSet { + const dnsResolveInterval = 10 * time.Second + + dnsProviderReg := extprom.WrapRegistererWithPrefix("cortex_chunksgateway_client_", reg) + + reg = extprom.WrapRegistererWithPrefix("cortex_chunksgateway_client_", reg) + + s := &chunksStoreBalancedSet{ + serviceAddresses: serviceAddresses, + dnsProvider: dns.NewProvider(logger, dnsProviderReg, dns.GolangResolverType), + clientsPool: newChunksGatewayClientPool(nil, clientConfig, logger, reg), + logger: logger, + } + + s.Service = services.NewTimerService(dnsResolveInterval, s.starting, s.resolve, nil) + return s +} + +func (s *chunksStoreBalancedSet) starting(ctx context.Context) error { + // Initial DNS resolution. + return s.resolve(ctx) +} + +func (s *chunksStoreBalancedSet) resolve(ctx context.Context) error { + if err := s.dnsProvider.Resolve(ctx, s.serviceAddresses); err != nil { + level.Error(s.logger).Log("msg", "failed to resolve store-gateway addresses", "err", err, "addresses", s.serviceAddresses) + } + return nil +} + +func (s *chunksStoreBalancedSet) GetClientsFor(_ string, blockIDs []ulid.ULID, exclude map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[ulid.ULID]ChunksStoreClient, error) { + addresses := s.dnsProvider.Addresses() + if len(addresses) == 0 { + return nil, fmt.Errorf("no address resolved for the store-gateway service addresses %s", strings.Join(s.serviceAddresses, ",")) + } + + // Randomize the list of addresses to not always query the same address. + rand.Shuffle(len(addresses), func(i, j int) { + addresses[i], addresses[j] = addresses[j], addresses[i] + }) + + // Pick a non excluded client for each block. + clients := map[ulid.ULID]ChunksStoreClient{} + + for _, blockID := range blockIDs { + // Pick the first non excluded store-gateway instance. + addr := getFirstNonExcludedAddr(addresses, exclude[blockID]) + if addr == "" { + return nil, fmt.Errorf("no store-gateway instance left after filtering out excluded instances for block %s", blockID.String()) + } + + c, err := s.clientsPool.GetClientFor(addr) + if err != nil { + return nil, errors.Wrapf(err, "failed to get store-gateway client for %s", addr) + } + + clients[blockID] = c.(ChunksStoreClient) + } + + return clients, nil +} diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 1455dbe37e..cdbcd0d423 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -74,8 +74,9 @@ type Config struct { LookbackDelta time.Duration `yaml:"lookback_delta"` // Blocks storage only. - StoreGatewayAddresses string `yaml:"store_gateway_addresses"` - StoreGatewayClient ClientConfig `yaml:"store_gateway_client"` + StoreGatewayAddresses string `yaml:"store_gateway_addresses"` + ChunksGatewayAddresses string `yaml:"chunks_gateway_addresses"` + StoreGatewayClient ClientConfig `yaml:"store_gateway_client"` ShuffleShardingIngestersLookbackPeriod time.Duration `yaml:"shuffle_sharding_ingesters_lookback_period"` @@ -111,6 +112,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 0, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. When running the blocks storage, if this option is enabled, the time range of the query sent to the store will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") f.StringVar(&cfg.ActiveQueryTrackerDir, "querier.active-query-tracker-dir", "./active-query-tracker", "Active query tracker monitors active queries, and writes them to the file in given directory. If Cortex discovers any queries in this log during startup, it will log them to the log file. Setting to empty value disables active query tracker, which also disables -querier.max-concurrent option.") f.StringVar(&cfg.StoreGatewayAddresses, "querier.store-gateway-addresses", "", "Comma separated list of store-gateway addresses in DNS Service Discovery format. This option should be set when using the blocks storage and the store-gateway sharding is disabled (when enabled, the store-gateway instances form a ring and addresses are picked from the ring).") + f.StringVar(&cfg.ChunksGatewayAddresses, "querier.chunks-gateway-addresses", "", "Comma separated list of chunks-gateway addresses in DNS Service Discovery format. This option should be set when using the blocks storage and the store-gateway sharding is disabled (when enabled, the store-gateway instances form a ring and addresses are picked from the ring).") f.DurationVar(&cfg.LookbackDelta, "querier.lookback-delta", 5*time.Minute, "Time since the last sample after which a time series is considered stale and ignored by expression evaluations.") f.DurationVar(&cfg.ShuffleShardingIngestersLookbackPeriod, "querier.shuffle-sharding-ingesters-lookback-period", 0, "When distributor's sharding strategy is shuffle-sharding and this setting is > 0, queriers fetch in-memory series from the minimum set of required ingesters, selecting only ingesters which may have received series since 'now - lookback period'. The lookback period should be greater or equal than the configured 'query store after' and 'query ingesters within'. If this setting is 0, queriers always query all ingesters (ingesters shuffle sharding on read path is disabled).") f.BoolVar(&cfg.ThanosEngine, "querier.thanos-engine", false, "Experimental. Use Thanos promql engine https://github.com/thanos-io/promql-engine rather than the Prometheus promql engine.") @@ -142,6 +144,14 @@ func (cfg *Config) GetStoreGatewayAddresses() []string { return strings.Split(cfg.StoreGatewayAddresses, ",") } +func (cfg *Config) GetChunksGatewayAddresses() []string { + if cfg.ChunksGatewayAddresses == "" { + return nil + } + + return strings.Split(cfg.ChunksGatewayAddresses, ",") +} + func getChunksIteratorFunction(cfg Config) chunkIteratorFunc { if cfg.BatchIterators { return batch.NewChunkMergeIterator diff --git a/pkg/querier/series_set.go b/pkg/querier/series_set.go new file mode 100644 index 0000000000..29e800cd8a --- /dev/null +++ b/pkg/querier/series_set.go @@ -0,0 +1,72 @@ +package querier + +import ( + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/go-kit/log" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +// Implementation of storage.SeriesSet, based on individual responses from store client. +type blockSeriesSet struct { + logger log.Logger + series []*typespb.SelectedSeries + chunks [][]*typespb.AggrChunk + warnings annotations.Annotations + + // next response to process + next int + + currSeries storage.Series +} + +func (bqss *blockSeriesSet) Next() bool { + bqss.currSeries = nil + + if bqss.next >= len(bqss.series) { + return false + } + + currLabels := labelpb.ZLabelsToPromLabels(bqss.series[bqss.next].Labels) + currChunks := bqss.chunks[bqss.next] + + bqss.next++ + + // Merge chunks for current series. Chunks may come in multiple responses, but as soon + // as the response has chunks for a new series, we can stop searching. Series are sorted. + // See documentation for StoreClient.Series call for details. + for bqss.next < len(bqss.series) && labels.Compare(currLabels, labelpb.ZLabelsToPromLabels(bqss.series[bqss.next].Labels)) == 0 { + currChunks = append(currChunks, bqss.chunks[bqss.next]...) + bqss.next++ + } + + chunks := make([]typespb.AggrChunk, 0, len(currChunks)) + for _, c := range currChunks { + chunks = append(chunks, *c) + } + + // level.Info(bqss.logger).Log("msg", "Inside blockSeriesSet.Next", + // "labels", currLabels, + // "chunks", len(currChunks), + // ) + bqss.currSeries = newBlockQuerierSeries(currLabels, chunks) + return true +} + +func (bqss *blockSeriesSet) At() storage.Series { + // level.Info(bqss.logger).Log("msg", "Inside blockSeriesSet.At", + // "series", bqss.currSeries.Labels(), + // ) + return bqss.currSeries +} + +func (bqss *blockSeriesSet) Err() error { + return nil +} + +func (bqss *blockSeriesSet) Warnings() annotations.Annotations { + return bqss.warnings +} diff --git a/pkg/querier/store_gateway_client.go b/pkg/querier/store_gateway_client.go index 6800fc0382..2ee7f2ec85 100644 --- a/pkg/querier/store_gateway_client.go +++ b/pkg/querier/store_gateway_client.go @@ -31,6 +31,20 @@ func newStoreGatewayClientFactory(clientCfg grpcclient.Config, reg prometheus.Re } } +func newChunksGatewayClientFactory(clientCfg grpcclient.Config, reg prometheus.Registerer) client.PoolFactory { + requestDuration := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "cortex", + Name: "chunks_client_request_duration_seconds", + Help: "Time spent executing requests to the store-gateway.", + Buckets: prometheus.ExponentialBuckets(0.008, 4, 7), + ConstLabels: prometheus.Labels{"client": "querier"}, + }, []string{"operation", "status_code"}) + + return func(addr string) (client.PoolClient, error) { + return dialChunksGatewayClient(clientCfg, addr, requestDuration) + } +} + func dialStoreGatewayClient(clientCfg grpcclient.Config, addr string, requestDuration *prometheus.HistogramVec) (*storeGatewayClient, error) { opts, err := clientCfg.DialOption(grpcclient.Instrument(requestDuration)) if err != nil { @@ -49,6 +63,42 @@ func dialStoreGatewayClient(clientCfg grpcclient.Config, addr string, requestDur }, nil } +func dialChunksGatewayClient(clientCfg grpcclient.Config, addr string, requestDuration *prometheus.HistogramVec) (*chunksGatewayClient, error) { + opts, err := clientCfg.DialOption(grpcclient.Instrument(requestDuration)) + if err != nil { + return nil, err + } + + conn, err := grpc.Dial(addr, opts...) + if err != nil { + return nil, errors.Wrapf(err, "failed to dial store-gateway %s", addr) + } + + return &chunksGatewayClient{ + ChunksGatewayClient: storegatewaypb.NewChunksGatewayClient(conn), + HealthClient: grpc_health_v1.NewHealthClient(conn), + conn: conn, + }, nil +} + +type chunksGatewayClient struct { + storegatewaypb.ChunksGatewayClient + grpc_health_v1.HealthClient + conn *grpc.ClientConn +} + +func (c *chunksGatewayClient) Close() error { + return c.conn.Close() +} + +func (c *chunksGatewayClient) String() string { + return c.RemoteAddress() +} + +func (c *chunksGatewayClient) RemoteAddress() string { + return c.conn.Target() +} + type storeGatewayClient struct { storegatewaypb.StoreGatewayClient grpc_health_v1.HealthClient @@ -95,6 +145,34 @@ func newStoreGatewayClientPool(discovery client.PoolServiceDiscovery, clientConf return client.NewPool("store-gateway", poolCfg, discovery, newStoreGatewayClientFactory(clientCfg, reg), clientsCount, logger) } +func newChunksGatewayClientPool(discovery client.PoolServiceDiscovery, clientConfig ClientConfig, logger log.Logger, reg prometheus.Registerer) *client.Pool { + // We prefer sane defaults instead of exposing further config options. + clientCfg := grpcclient.Config{ + MaxRecvMsgSize: 100 << 20, + MaxSendMsgSize: 16 << 20, + GRPCCompression: clientConfig.GRPCCompression, + RateLimit: 0, + RateLimitBurst: 0, + BackoffOnRatelimits: false, + TLSEnabled: clientConfig.TLSEnabled, + TLS: clientConfig.TLS, + } + poolCfg := client.PoolConfig{ + CheckInterval: time.Minute, + HealthCheckEnabled: true, + HealthCheckTimeout: 10 * time.Second, + } + + clientsCount := promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Namespace: "cortex", + Name: "storegateway_clients", + Help: "The current number of store-gateway clients in the pool.", + ConstLabels: map[string]string{"client": "querier"}, + }) + + return client.NewPool("store-gateway", poolCfg, discovery, newChunksGatewayClientFactory(clientCfg, reg), clientsCount, logger) +} + type ClientConfig struct { TLSEnabled bool `yaml:"tls_enabled"` TLS tls.ClientConfig `yaml:",inline"` diff --git a/pkg/querier/store_gateway_client_test.go b/pkg/querier/store_gateway_client_test.go index c1b306a7e1..69171f03ae 100644 --- a/pkg/querier/store_gateway_client_test.go +++ b/pkg/querier/store_gateway_client_test.go @@ -5,11 +5,11 @@ import ( "net" "testing" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/prometheus/client_golang/prometheus" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/user" "google.golang.org/grpc" @@ -74,6 +74,10 @@ func (m *mockStoreGatewayServer) Series(_ *storepb.SeriesRequest, srv storegatew return nil } +func (m *mockStoreGatewayServer) Select(_ *storepb.SelectRequest, srv storegatewaypb.StoreGateway_SelectServer) error { + return nil +} + func (m *mockStoreGatewayServer) LabelNames(context.Context, *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { return nil, nil } diff --git a/pkg/querier/tripperware/shard_by.go b/pkg/querier/tripperware/shard_by.go index 39bef61ca9..4f9494385f 100644 --- a/pkg/querier/tripperware/shard_by.go +++ b/pkg/querier/tripperware/shard_by.go @@ -4,10 +4,10 @@ import ( "context" "net/http" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/thanos-io/thanos/pkg/querysharding" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/httpgrpc" querier_stats "github.com/cortexproject/cortex/pkg/querier/stats" diff --git a/pkg/querier/tripperware/test_shard_by_query_utils.go b/pkg/querier/tripperware/test_shard_by_query_utils.go index 657d7daa3a..a565343d38 100644 --- a/pkg/querier/tripperware/test_shard_by_query_utils.go +++ b/pkg/querier/tripperware/test_shard_by_query_utils.go @@ -12,12 +12,12 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/promql/parser" "github.com/stretchr/testify/require" thanosquerysharding "github.com/thanos-io/thanos/pkg/querysharding" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/user" "github.com/cortexproject/cortex/pkg/querysharding" diff --git a/pkg/querysharding/util.go b/pkg/querysharding/util.go index 3f2a2d8243..7e19748854 100644 --- a/pkg/querysharding/util.go +++ b/pkg/querysharding/util.go @@ -4,9 +4,9 @@ import ( "encoding/base64" "sync" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql/parser" - "github.com/thanos-io/thanos/pkg/store/storepb" ) const ( diff --git a/pkg/storegateway/block_chunks_client.go b/pkg/storegateway/block_chunks_client.go new file mode 100644 index 0000000000..e67ee26a17 --- /dev/null +++ b/pkg/storegateway/block_chunks_client.go @@ -0,0 +1,66 @@ +package storegateway + +import ( + "context" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/go-kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/thanos/pkg/runutil" +) + +// Loads a batch of chunks +type blockChunkClient struct { + ctx context.Context + logger log.Logger + + chunkr *bucketChunkReader + loadAggregates []storepb.Aggr + bytesLimiter BytesLimiter + + calculateChunkHash bool + tenant string +} + +func newBlockChunkClient( + ctx context.Context, + logger log.Logger, + b *bucketBlock, + limiter ChunksLimiter, + bytesLimiter BytesLimiter, + calculateChunkHash bool, + chunkFetchDuration *prometheus.HistogramVec, + chunkFetchDurationSum *prometheus.HistogramVec, +) *blockChunkClient { + chunkr := b.chunkReader() + return &blockChunkClient{ + ctx: ctx, + logger: logger, + chunkr: chunkr, + bytesLimiter: bytesLimiter, + calculateChunkHash: calculateChunkHash, + } +} + +func (b *blockChunkClient) loadChunks(entries []seriesEntry) error { + b.chunkr.reset() + + for i, s := range entries { + for j := range s.chks { + if err := b.chunkr.addLoad(s.refs[j], i, j); err != nil { + return errors.Wrap(err, "add chunk load") + } + } + } + + if err := b.chunkr.load(b.ctx, entries, b.loadAggregates, b.calculateChunkHash, b.bytesLimiter, b.tenant); err != nil { + return errors.Wrap(err, "load chunks") + } + + return nil +} + +func (b *blockChunkClient) Close() { + runutil.CloseWithLogOnErr(b.logger, b.chunkr, "series block") +} diff --git a/pkg/storegateway/block_index_client.go b/pkg/storegateway/block_index_client.go new file mode 100644 index 0000000000..01c5a51f35 --- /dev/null +++ b/pkg/storegateway/block_index_client.go @@ -0,0 +1,255 @@ +package storegateway + +import ( + "context" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/go-kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/thanos-io/thanos/pkg/runutil" + "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/weaveworks/common/httpgrpc" + "google.golang.org/grpc/codes" +) + +type blockIndexRequest struct { + maxTime int64 + minTime int64 + skipChunks bool +} + +type blockIndexClient struct { + ctx context.Context + logger log.Logger + extLset labels.Labels + extLsetToRemove map[string]struct{} + + mint int64 + maxt int64 + indexr *bucketIndexReader + bytesLimiter BytesLimiter + lazyExpandedPostingEnabled bool + lazyExpandedPostingsCount prometheus.Counter + lazyExpandedPostingSizeBytes prometheus.Counter + lazyExpandedPostingSeriesOverfetchedSizeBytes prometheus.Counter + + skipChunks bool + shardMatcher *storepb.ShardMatcher + blockMatchers []*labels.Matcher + + seriesFetchDurationSum *prometheus.HistogramVec + + tenant string + + // Internal state. + i uint64 + lazyPostings *lazyExpandedPostings + expandedPostings []storage.SeriesRef + chkMetas []chunks.Meta + lset labels.Labels + symbolizedLset []symbolizedLabel + entries []seriesEntry + hasMorePostings bool + batchSize int +} + +func newBlockIndexClient( + ctx context.Context, + logger log.Logger, + b *bucketBlock, + req blockIndexRequest, + bytesLimiter BytesLimiter, + blockMatchers []*labels.Matcher, + shardMatcher *storepb.ShardMatcher, + batchSize int, + seriesFetchDurationSum *prometheus.HistogramVec, + extLsetToRemove map[string]struct{}, + lazyExpandedPostingEnabled bool, + lazyExpandedPostingsCount prometheus.Counter, + lazyExpandedPostingSizeBytes prometheus.Counter, + lazyExpandedPostingSeriesOverfetchedSizeBytes prometheus.Counter, + tenant string, +) *blockIndexClient { + extLset := b.extLset + if extLsetToRemove != nil { + extLset = rmLabels(extLset.Copy(), extLsetToRemove) + } + + return &blockIndexClient{ + ctx: ctx, + logger: logger, + extLset: extLset, + extLsetToRemove: extLsetToRemove, + + mint: req.minTime, + maxt: req.maxTime, + indexr: b.indexReader(), + bytesLimiter: bytesLimiter, + skipChunks: req.skipChunks, + seriesFetchDurationSum: seriesFetchDurationSum, + + lazyExpandedPostingEnabled: lazyExpandedPostingEnabled, + lazyExpandedPostingsCount: lazyExpandedPostingsCount, + lazyExpandedPostingSizeBytes: lazyExpandedPostingSizeBytes, + lazyExpandedPostingSeriesOverfetchedSizeBytes: lazyExpandedPostingSeriesOverfetchedSizeBytes, + + shardMatcher: shardMatcher, + blockMatchers: blockMatchers, + hasMorePostings: true, + batchSize: batchSize, + tenant: tenant, + } +} + +func (b *blockIndexClient) ExpandPostings( + matchers sortedMatchers, + seriesLimiter SeriesLimiter, +) error { + ps, err := b.indexr.ExpandedPostings(b.ctx, matchers, b.bytesLimiter, b.lazyExpandedPostingEnabled, b.lazyExpandedPostingSizeBytes, b.tenant) + if err != nil { + return errors.Wrap(err, "expanded matching posting") + } + + if ps == nil || len(ps.postings) == 0 { + b.lazyPostings = emptyLazyPostings + return nil + } + b.lazyPostings = ps + + // If lazy expanded posting enabled, it is possible to fetch more series + // so easier to hit the series limit. + if err := seriesLimiter.Reserve(uint64(len(ps.postings))); err != nil { + return httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded series limit: %s", err) + } + + if b.batchSize > len(ps.postings) { + b.batchSize = len(ps.postings) + } + if b.lazyPostings.lazyExpanded() { + // Assume lazy expansion could cut actual expanded postings length to 50%. + b.expandedPostings = make([]storage.SeriesRef, 0, len(b.lazyPostings.postings)/2) + b.lazyExpandedPostingsCount.Inc() + } + b.entries = make([]seriesEntry, 0, b.batchSize) + return nil +} + +func (b *blockIndexClient) MergeStats(stats *queryStats) *queryStats { + stats.merge(b.indexr.stats) + return stats +} + +func (b *blockIndexClient) Close() { + runutil.CloseWithLogOnErr(b.logger, b.indexr, "series block") +} + +func (b *blockIndexClient) hasNext() bool { + return b.hasMorePostings +} + +func (b *blockIndexClient) nextBatch(tenant string) ([]seriesEntry, error) { + start := b.i + end := start + uint64(b.batchSize) + if end > uint64(len(b.lazyPostings.postings)) { + end = uint64(len(b.lazyPostings.postings)) + } + b.i = end + + postingsBatch := b.lazyPostings.postings[start:end] + if len(postingsBatch) == 0 { + b.hasMorePostings = false + if b.lazyPostings.lazyExpanded() { + v, err := b.indexr.IndexVersion() + if err != nil { + return nil, errors.Wrap(err, "get index version") + } + if v >= 2 { + for i := range b.expandedPostings { + b.expandedPostings[i] = b.expandedPostings[i] / 16 + } + } + b.indexr.storeExpandedPostingsToCache(b.blockMatchers, index.NewListPostings(b.expandedPostings), len(b.expandedPostings), tenant) + } + return nil, nil + } + + b.indexr.reset(len(postingsBatch)) + + if err := b.indexr.PreloadSeries(b.ctx, postingsBatch, b.bytesLimiter, b.tenant); err != nil { + return nil, errors.Wrap(err, "preload series") + } + + b.entries = b.entries[:0] +OUTER: + for i := 0; i < len(postingsBatch); i++ { + if err := b.ctx.Err(); err != nil { + return nil, err + } + ok, err := b.indexr.LoadSeriesForTime(postingsBatch[i], &b.symbolizedLset, &b.chkMetas, b.skipChunks, b.mint, b.maxt) + if err != nil { + return nil, errors.Wrap(err, "read series") + } + if !ok { + continue + } + + if err := b.indexr.LookupLabelsSymbols(b.ctx, b.symbolizedLset, &b.lset); err != nil { + return nil, errors.Wrap(err, "Lookup labels symbols") + } + + for _, matcher := range b.lazyPostings.matchers { + val := b.lset.Get(matcher.Name) + if !matcher.Matches(val) { + // Series not matched means series we overfetched due to lazy posting expansion. + seriesBytes := b.indexr.loadedSeries[postingsBatch[i]] + b.lazyExpandedPostingSeriesOverfetchedSizeBytes.Add(float64(len(seriesBytes))) + continue OUTER + } + } + if b.lazyPostings.lazyExpanded() { + b.expandedPostings = append(b.expandedPostings, postingsBatch[i]) + } + + completeLabelset := labelpb.ExtendSortedLabels(b.lset, b.extLset) + if b.extLsetToRemove != nil { + completeLabelset = rmLabels(completeLabelset, b.extLsetToRemove) + } + + if !b.shardMatcher.MatchesLabels(completeLabelset) { + continue + } + + s := seriesEntry{lset: completeLabelset} + if b.skipChunks { + b.entries = append(b.entries, s) + continue + } + + // Schedule loading chunks. + s.refs = make([]chunks.ChunkRef, 0, len(b.chkMetas)) + s.chks = make([]typespb.AggrChunk, 0, len(b.chkMetas)) + + for _, meta := range b.chkMetas { + s.meta = append(s.meta, &typespb.ChunkMeta{ + Mint: meta.MinTime, + Maxt: meta.MaxTime, + Ref: uint64(meta.Ref), + }) + s.refs = append(s.refs, meta.Ref) + s.chks = append(s.chks, typespb.AggrChunk{ + MinTime: meta.MinTime, + MaxTime: meta.MaxTime, + }) + } + + b.entries = append(b.entries, s) + } + + return b.entries, nil +} diff --git a/pkg/storegateway/block_series_client.go b/pkg/storegateway/block_series_client.go new file mode 100644 index 0000000000..69242cb228 --- /dev/null +++ b/pkg/storegateway/block_series_client.go @@ -0,0 +1,220 @@ +package storegateway + +import ( + "context" + "io" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/thanos-io/thanos/pkg/runutil" + "github.com/thanos-io/thanos/pkg/store/labelpb" + "google.golang.org/grpc" +) + +type seriesEntry struct { + lset labels.Labels + meta []*typespb.ChunkMeta + refs []chunks.ChunkRef + chks []typespb.AggrChunk +} + +// blockSeriesClient is a storepb.Store_SeriesClient for a +// single TSDB block in object storage. +type blockSeriesClient struct { + grpc.ClientStream + ctx context.Context + logger log.Logger + extLset labels.Labels + extLsetToRemove map[string]struct{} + + mint int64 + maxt int64 + indexr *bucketIndexReader + chunkr *bucketChunkReader + loadAggregates []storepb.Aggr + chunksLimiter ChunksLimiter + bytesLimiter BytesLimiter + + lazyExpandedPostingEnabled bool + lazyExpandedPostingsCount prometheus.Counter + lazyExpandedPostingSizeBytes prometheus.Counter + lazyExpandedPostingSeriesOverfetchedSizeBytes prometheus.Counter + + skipChunks bool + shardMatcher *storepb.ShardMatcher + blockMatchers []*labels.Matcher + calculateChunkHash bool + seriesFetchDurationSum *prometheus.HistogramVec + chunkFetchDuration *prometheus.HistogramVec + chunkFetchDurationSum *prometheus.HistogramVec + tenant string + + entries []seriesEntry + batchSize int + + // New stuff + blockIndexClient *blockIndexClient + blockChunkClient *blockChunkClient +} + +func newBlockSeriesClient( + ctx context.Context, + logger log.Logger, + b *bucketBlock, + req *storepb.SeriesRequest, + limiter ChunksLimiter, + bytesLimiter BytesLimiter, + blockMatchers []*labels.Matcher, + shardMatcher *storepb.ShardMatcher, + calculateChunkHash bool, + batchSize int, + seriesFetchDurationSum *prometheus.HistogramVec, + chunkFetchDuration *prometheus.HistogramVec, + chunkFetchDurationSum *prometheus.HistogramVec, + extLsetToRemove map[string]struct{}, + lazyExpandedPostingEnabled bool, + lazyExpandedPostingsCount prometheus.Counter, + lazyExpandedPostingSizeBytes prometheus.Counter, + lazyExpandedPostingSeriesOverfetchedSizeBytes prometheus.Counter, + tenant string, +) *blockSeriesClient { + var chunkr *bucketChunkReader + if !req.SkipChunks { + chunkr = b.chunkReader() + } + + extLset := b.extLset + if extLsetToRemove != nil { + extLset = rmLabels(extLset.Copy(), extLsetToRemove) + } + + blockIndexClient := newBlockIndexClient( + ctx, + logger, + b, + blockIndexRequest{minTime: req.MinTime, maxTime: req.MaxTime}, + bytesLimiter, + blockMatchers, + shardMatcher, + batchSize, + seriesFetchDurationSum, + extLsetToRemove, + lazyExpandedPostingEnabled, + lazyExpandedPostingsCount, + lazyExpandedPostingSizeBytes, + lazyExpandedPostingSeriesOverfetchedSizeBytes, + tenant, + ) + + blockChunkClient := newBlockChunkClient( + ctx, + logger, + b, + limiter, + bytesLimiter, + calculateChunkHash, + chunkFetchDuration, + chunkFetchDurationSum, + ) + + return &blockSeriesClient{ + ctx: ctx, + logger: logger, + extLset: extLset, + extLsetToRemove: extLsetToRemove, + + mint: req.MinTime, + maxt: req.MaxTime, + indexr: b.indexReader(), + chunkr: chunkr, + chunksLimiter: limiter, + bytesLimiter: bytesLimiter, + skipChunks: req.SkipChunks, + seriesFetchDurationSum: seriesFetchDurationSum, + chunkFetchDuration: chunkFetchDuration, + chunkFetchDurationSum: chunkFetchDurationSum, + + lazyExpandedPostingEnabled: lazyExpandedPostingEnabled, + lazyExpandedPostingsCount: lazyExpandedPostingsCount, + lazyExpandedPostingSizeBytes: lazyExpandedPostingSizeBytes, + lazyExpandedPostingSeriesOverfetchedSizeBytes: lazyExpandedPostingSeriesOverfetchedSizeBytes, + + loadAggregates: req.Aggregates, + shardMatcher: shardMatcher, + blockMatchers: blockMatchers, + calculateChunkHash: calculateChunkHash, + batchSize: batchSize, + tenant: tenant, + + // New + blockIndexClient: blockIndexClient, + blockChunkClient: blockChunkClient, + } +} + +func (b *blockSeriesClient) Close() { + if !b.skipChunks { + runutil.CloseWithLogOnErr(b.logger, b.chunkr, "series block") + } + + b.blockIndexClient.Close() +} + +func (b *blockSeriesClient) MergeStats(stats *queryStats) *queryStats { + b.blockIndexClient.MergeStats(stats) + if !b.skipChunks { + stats.merge(b.chunkr.stats) + } + return stats +} + +func (b *blockSeriesClient) ExpandPostings( + matchers sortedMatchers, + seriesLimiter SeriesLimiter, +) error { + return b.blockIndexClient.ExpandPostings(matchers, seriesLimiter) +} + +func (b *blockSeriesClient) Recv() (*storepb.SeriesResponse, error) { + for len(b.entries) == 0 && b.blockIndexClient.hasNext() { + if err := b.nextBatch(b.tenant); err != nil { + return nil, err + } + } + + if len(b.entries) == 0 { + b.seriesFetchDurationSum.WithLabelValues(b.tenant).Observe(b.indexr.stats.SeriesDownloadLatencySum.Seconds()) + if b.chunkr != nil { + b.chunkFetchDuration.WithLabelValues(b.tenant).Observe(b.chunkr.stats.ChunksFetchDurationSum.Seconds()) + b.chunkFetchDurationSum.WithLabelValues(b.tenant).Observe(b.chunkr.stats.ChunksDownloadLatencySum.Seconds()) + } + return nil, io.EOF + } + + next := b.entries[0] + b.entries = b.entries[1:] + + return storepb.NewSeriesResponse(&typespb.Series{ + Labels: labelpb.ZLabelsFromPromLabels(next.lset), + Chunks: next.chks, + }), nil +} + +func (b *blockSeriesClient) nextBatch(tenant string) error { + seriesEntries, err := b.blockIndexClient.nextBatch(tenant) + if err != nil { + return err + } + + b.entries = seriesEntries + + if b.skipChunks { + return nil + } + + return b.blockChunkClient.loadChunks(b.entries) +} diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go new file mode 100644 index 0000000000..46777c2099 --- /dev/null +++ b/pkg/storegateway/bucket.go @@ -0,0 +1,3343 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "bufio" + "bytes" + "context" + "encoding/binary" + "fmt" + "hash" + "io" + "math" + "os" + "path" + "path/filepath" + "sort" + "strings" + "sync" + "time" + + "github.com/alecthomas/units" + "github.com/cespare/xxhash" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/gogo/protobuf/types" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/encoding" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/weaveworks/common/httpgrpc" + "golang.org/x/exp/slices" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/thanos-io/objstore" + + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/indexheader" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact/downsample" + "github.com/thanos-io/thanos/pkg/extprom" + "github.com/thanos-io/thanos/pkg/gate" + "github.com/thanos-io/thanos/pkg/info/infopb" + "github.com/thanos-io/thanos/pkg/model" + "github.com/thanos-io/thanos/pkg/pool" + "github.com/thanos-io/thanos/pkg/runutil" + storecache "github.com/thanos-io/thanos/pkg/store/cache" + "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/thanos-io/thanos/pkg/strutil" + "github.com/thanos-io/thanos/pkg/tenancy" + "github.com/thanos-io/thanos/pkg/tracing" + + "github.com/cortexproject/cortex/pkg/storegateway/hintspb" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" +) + +const ( + // MaxSamplesPerChunk is approximately the max number of samples that we may have in any given chunk. This is needed + // for precalculating the number of samples that we may have to retrieve and decode for any given query + // without downloading them. Please take a look at https://github.com/prometheus/tsdb/pull/397 to know + // where this number comes from. Long story short: TSDB is made in such a way, and it is made in such a way + // because you barely get any improvements in compression when the number of samples is beyond this. + // Take a look at Figure 6 in this whitepaper http://www.vldb.org/pvldb/vol8/p1816-teller.pdf. + MaxSamplesPerChunk = 120 + // EstimatedMaxChunkSize is average max of chunk size. This can be exceeded though in very rare (valid) cases. + EstimatedMaxChunkSize = 16000 + EstimatedMaxSeriesSize = 64 * 1024 + // Relatively large in order to reduce memory waste, yet small enough to avoid excessive allocations. + chunkBytesPoolMinSize = 64 * 1024 // 64 KiB + chunkBytesPoolMaxSize = 64 * 1024 * 1024 // 64 MiB + + // CompatibilityTypeLabelName is an artificial label that Store Gateway can optionally advertise. This is required for compatibility + // with pre v0.8.0 Querier. Previous Queriers was strict about duplicated external labels of all StoreAPIs that had any labels. + // Now with newer Store Gateway advertising all the external labels it has access to, there was simple case where + // Querier was blocking Store Gateway as duplicate with sidecar. + // + // Newer Queriers are not strict, no duplicated external labels check is there anymore. + // Additionally newer Queriers removes/ignore this exact labels from UI and querying. + // + // This label name is intentionally against Prometheus label style. + // TODO(bwplotka): Remove it at some point. + CompatibilityTypeLabelName = "@thanos_compatibility_store_type" + + // DefaultPostingOffsetInMemorySampling represents default value for --store.index-header-posting-offsets-in-mem-sampling. + // 32 value is chosen as it's a good balance for common setups. Sampling that is not too large (too many CPU cycles) and + // not too small (too much memory). + DefaultPostingOffsetInMemorySampling = 32 + + PartitionerMaxGapSize = 512 * 1024 + + // Labels for metrics. + labelEncode = "encode" + labelDecode = "decode" + + minBlockSyncConcurrency = 1 + + enableChunkHashCalculation = true + + // SeriesBatchSize is the default batch size when fetching series from object storage. + SeriesBatchSize = 10000 + + initialBufSize = 32 * 1024 // 32KB seems like a good minimum starting size for sync pool size. +) + +var ( + errBlockSyncConcurrencyNotValid = errors.New("the block sync concurrency must be equal or greater than 1.") + hashPool = sync.Pool{New: func() interface{} { return xxhash.New() }} +) + +type bucketStoreMetrics struct { + blocksLoaded prometheus.Gauge + blockLoads prometheus.Counter + blockLoadFailures prometheus.Counter + lastLoadedBlock prometheus.Gauge + blockDrops prometheus.Counter + blockDropFailures prometheus.Counter + blockLoadDuration prometheus.Histogram + seriesDataTouched *prometheus.HistogramVec + seriesDataFetched *prometheus.HistogramVec + seriesDataSizeTouched *prometheus.HistogramVec + seriesDataSizeFetched *prometheus.HistogramVec + seriesBlocksQueried *prometheus.HistogramVec + seriesGetAllDuration *prometheus.HistogramVec + seriesMergeDuration *prometheus.HistogramVec + resultSeriesCount *prometheus.HistogramVec + chunkSizeBytes *prometheus.HistogramVec + postingsSizeBytes *prometheus.HistogramVec + queriesDropped *prometheus.CounterVec + seriesRefetches *prometheus.CounterVec + chunkRefetches *prometheus.CounterVec + emptyPostingCount *prometheus.CounterVec + + lazyExpandedPostingsCount prometheus.Counter + lazyExpandedPostingSizeBytes prometheus.Counter + lazyExpandedPostingSeriesOverfetchedSizeBytes prometheus.Counter + + cachedPostingsCompressions *prometheus.CounterVec + cachedPostingsCompressionErrors *prometheus.CounterVec + cachedPostingsCompressionTimeSeconds *prometheus.CounterVec + cachedPostingsOriginalSizeBytes *prometheus.CounterVec + cachedPostingsCompressedSizeBytes *prometheus.CounterVec + + seriesFetchDuration *prometheus.HistogramVec + // Counts time for fetching series across all batches. + seriesFetchDurationSum *prometheus.HistogramVec + postingsFetchDuration *prometheus.HistogramVec + // chunkFetchDuration counts total time loading chunks, but since we spawn + // multiple goroutines the actual latency is usually much lower than it. + chunkFetchDuration *prometheus.HistogramVec + // Actual absolute total time for loading chunks. + chunkFetchDurationSum *prometheus.HistogramVec +} + +func newBucketStoreMetrics(reg prometheus.Registerer) *bucketStoreMetrics { + var m bucketStoreMetrics + + m.blockLoads = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_block_loads_total", + Help: "Total number of remote block loading attempts.", + }) + m.blockLoadFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_block_load_failures_total", + Help: "Total number of failed remote block loading attempts.", + }) + m.blockDrops = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_block_drops_total", + Help: "Total number of local blocks that were dropped.", + }) + m.blockDropFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_block_drop_failures_total", + Help: "Total number of local blocks that failed to be dropped.", + }) + m.blocksLoaded = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "thanos_bucket_store_blocks_loaded", + Help: "Number of currently loaded blocks.", + }) + m.lastLoadedBlock = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "thanos_bucket_store_blocks_last_loaded_timestamp_seconds", + Help: "Timestamp when last block got loaded.", + }) + m.blockLoadDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_block_load_duration_seconds", + Help: "The total time taken to load a block in seconds.", + Buckets: []float64{0.1, 0.5, 1, 10, 20, 30, 60, 120}, + }) + + m.seriesDataTouched = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_data_touched", + Help: "Number of items of a data type touched to fulfill a single Store API series request.", + Buckets: prometheus.ExponentialBuckets(200, 2, 15), + }, []string{"data_type", tenancy.MetricLabel}) + m.seriesDataFetched = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_data_fetched", + Help: "Number of items of a data type retrieved to fulfill a single Store API series request.", + Buckets: prometheus.ExponentialBuckets(200, 2, 15), + }, []string{"data_type", tenancy.MetricLabel}) + + m.seriesDataSizeTouched = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_data_size_touched_bytes", + Help: "Total size of items of a data type touched to fulfill a single Store API series request in Bytes.", + Buckets: prometheus.ExponentialBuckets(1024, 2, 15), + }, []string{"data_type", tenancy.MetricLabel}) + m.seriesDataSizeFetched = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_data_size_fetched_bytes", + Help: "Total size of items of a data type fetched to fulfill a single Store API series request in Bytes.", + Buckets: prometheus.ExponentialBuckets(1024, 2, 15), + }, []string{"data_type", tenancy.MetricLabel}) + + m.seriesBlocksQueried = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_blocks_queried", + Help: "Number of blocks in a bucket store that were touched to satisfy a query.", + Buckets: prometheus.ExponentialBuckets(1, 2, 10), + }, []string{tenancy.MetricLabel}) + m.seriesGetAllDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_get_all_duration_seconds", + Help: "Time it takes until all per-block prepares and loads for a query are finished.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + m.seriesMergeDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_merge_duration_seconds", + Help: "Time it takes to merge sub-results from all queried blocks into a single result.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + m.resultSeriesCount = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_result_series", + Help: "Number of series observed in the final result of a query.", + Buckets: prometheus.ExponentialBuckets(100, 2, 15), // From 100 to 1638400. + }, []string{tenancy.MetricLabel}) + + m.chunkSizeBytes = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_sent_chunk_size_bytes", + Help: "Size in bytes of the chunks for the single series, which is adequate to the gRPC message size sent to querier.", + Buckets: []float64{ + 32, 256, 512, 1024, 32 * 1024, 256 * 1024, 512 * 1024, 1024 * 1024, 32 * 1024 * 1024, 256 * 1024 * 1024, 512 * 1024 * 1024, + }, + }, []string{tenancy.MetricLabel}) + + m.postingsSizeBytes = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_postings_size_bytes", + Help: "Size in bytes of the postings for a single series call.", + Buckets: []float64{ + 32, 256, 512, 1024, 32 * 1024, 256 * 1024, 512 * 1024, 1024 * 1024, 32 * 1024 * 1024, 128 * 1024 * 1024, 256 * 1024 * 1024, 512 * 1024 * 1024, 768 * 1024 * 1024, 1024 * 1024 * 1024, + }, + }, []string{tenancy.MetricLabel}) + + m.queriesDropped = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_queries_dropped_total", + Help: "Number of queries that were dropped due to the limit.", + }, []string{"reason", tenancy.MetricLabel}) + m.seriesRefetches = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_series_refetches_total", + Help: "Total number of cases where configured estimated series bytes was not enough was to fetch series from index, resulting in refetch.", + }, []string{tenancy.MetricLabel}) + m.chunkRefetches = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_chunk_refetches_total", + Help: "Total number of cases where configured estimated chunk bytes was not enough was to fetch chunks from object store, resulting in refetch.", + }, []string{tenancy.MetricLabel}) + + m.cachedPostingsCompressions = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_cached_postings_compressions_total", + Help: "Number of postings compressions before storing to index cache.", + }, []string{"op", tenancy.MetricLabel}) + m.cachedPostingsCompressions.WithLabelValues(labelEncode, tenancy.DefaultTenant) + m.cachedPostingsCompressions.WithLabelValues(labelDecode, tenancy.DefaultTenant) + + m.cachedPostingsCompressionErrors = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_cached_postings_compression_errors_total", + Help: "Number of postings compression errors.", + }, []string{"op", tenancy.MetricLabel}) + m.cachedPostingsCompressionErrors.WithLabelValues(labelEncode, tenancy.DefaultTenant) + m.cachedPostingsCompressionErrors.WithLabelValues(labelDecode, tenancy.DefaultTenant) + + m.cachedPostingsCompressionTimeSeconds = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_cached_postings_compression_time_seconds_total", + Help: "Time spent compressing postings before storing them into postings cache.", + }, []string{"op", tenancy.MetricLabel}) + m.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelEncode, tenancy.DefaultTenant) + m.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelDecode, tenancy.DefaultTenant) + + m.cachedPostingsOriginalSizeBytes = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_cached_postings_original_size_bytes_total", + Help: "Original size of postings stored into cache.", + }, []string{tenancy.MetricLabel}) + m.cachedPostingsCompressedSizeBytes = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_cached_postings_compressed_size_bytes_total", + Help: "Compressed size of postings stored into cache.", + }, []string{tenancy.MetricLabel}) + + m.seriesFetchDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_fetch_duration_seconds", + Help: "The time it takes to fetch series to respond to a request sent to a store gateway. It includes both the time to fetch it from the cache and from storage in case of cache misses.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + + m.seriesFetchDurationSum = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_series_fetch_duration_sum_seconds", + Help: "The total time it takes to fetch series to respond to a request sent to a store gateway across all series batches. It includes both the time to fetch it from the cache and from storage in case of cache misses.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + + m.postingsFetchDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_postings_fetch_duration_seconds", + Help: "The time it takes to fetch postings to respond to a request sent to a store gateway. It includes both the time to fetch it from the cache and from storage in case of cache misses.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + + m.chunkFetchDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_chunks_fetch_duration_seconds", + Help: "The total time spent fetching chunks within a single request for one block.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + + m.chunkFetchDurationSum = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "thanos_bucket_store_chunks_fetch_duration_sum_seconds", + Help: "The total absolute time spent fetching chunks within a single request for one block.", + Buckets: []float64{0.001, 0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120}, + }, []string{tenancy.MetricLabel}) + + m.emptyPostingCount = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_bucket_store_empty_postings_total", + Help: "Total number of empty postings when fetching block series.", + }, []string{tenancy.MetricLabel}) + + m.lazyExpandedPostingsCount = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_lazy_expanded_postings_total", + Help: "Total number of times when lazy expanded posting optimization applies.", + }) + + m.lazyExpandedPostingSizeBytes = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_lazy_expanded_posting_size_bytes_total", + Help: "Total number of lazy posting group size in bytes.", + }) + + m.lazyExpandedPostingSeriesOverfetchedSizeBytes = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_bucket_store_lazy_expanded_posting_series_overfetched_size_bytes_total", + Help: "Total number of series size in bytes overfetched due to posting lazy expansion.", + }) + + return &m +} + +// FilterConfig is a configuration, which Store uses for filtering metrics based on time. +type FilterConfig struct { + MinTime, MaxTime model.TimeOrDurationValue +} + +type BlockEstimator func(meta metadata.Meta) uint64 + +// BucketStore implements the store API backed by a bucket. It loads all index +// files to local disk. +// +// NOTE: Bucket store reencodes postings using diff+varint+snappy when storing to cache. +// This makes them smaller, but takes extra CPU and memory. +// When used with in-memory cache, memory usage should decrease overall, thanks to postings being smaller. +type BucketStore struct { + logger log.Logger + reg prometheus.Registerer // TODO(metalmatze) remove and add via BucketStoreOption + metrics *bucketStoreMetrics + bkt objstore.InstrumentedBucketReader + fetcher block.MetadataFetcher + dir string + loadIndex bool + indexCache storecache.IndexCache + indexReaderPool *indexheader.ReaderPool + buffers sync.Pool + chunkPool pool.Bytes + seriesBatchSize int + + // Sets of blocks that have the same labels. They are indexed by a hash over their label set. + mtx sync.RWMutex + blocks map[ulid.ULID]*bucketBlock + blockSets map[uint64]*bucketBlockSet + + // Verbose enabled additional logging. + debugLogging bool + // Number of goroutines to use when syncing blocks from object storage. + blockSyncConcurrency int + + // Query gate which limits the maximum amount of concurrent queries. + queryGate gate.Gate + + // chunksLimiterFactory creates a new limiter used to limit the number of chunks fetched by each Series() call. + chunksLimiterFactory ChunksLimiterFactory + // seriesLimiterFactory creates a new limiter used to limit the number of touched series by each Series() call, + // or LabelName and LabelValues calls when used with matchers. + seriesLimiterFactory SeriesLimiterFactory + + // bytesLimiterFactory creates a new limiter used to limit the amount of bytes fetched/touched by each Series() call. + bytesLimiterFactory BytesLimiterFactory + partitioner Partitioner + + filterConfig *FilterConfig + advLabelSets []labelpb.ZLabelSet + enableCompatibilityLabel bool + + // Every how many posting offset entry we pool in heap memory. Default in Prometheus is 32. + postingOffsetsInMemSampling int + + // Enables hints in the Series() response. + enableSeriesResponseHints bool + + enableChunkHashCalculation bool + + enabledLazyExpandedPostings bool + + sortingStrategy sortingStrategy + + blockEstimatedMaxSeriesFunc BlockEstimator + blockEstimatedMaxChunkFunc BlockEstimator +} + +func (s *BucketStore) validate() error { + if s.blockSyncConcurrency < minBlockSyncConcurrency { + return errBlockSyncConcurrencyNotValid + } + return nil +} + +type noopCache struct{} + +func (noopCache) StorePostings(ulid.ULID, labels.Label, []byte, string) {} +func (noopCache) FetchMultiPostings(_ context.Context, _ ulid.ULID, keys []labels.Label, tenant string) (map[labels.Label][]byte, []labels.Label) { + return map[labels.Label][]byte{}, keys +} + +func (noopCache) StoreExpandedPostings(_ ulid.ULID, _ []*labels.Matcher, _ []byte, tenant string) {} +func (noopCache) FetchExpandedPostings(_ context.Context, _ ulid.ULID, _ []*labels.Matcher, tenant string) ([]byte, bool) { + return []byte{}, false +} + +func (noopCache) StoreSeries(ulid.ULID, storage.SeriesRef, []byte, string) {} +func (noopCache) FetchMultiSeries(_ context.Context, _ ulid.ULID, ids []storage.SeriesRef, tenant string) (map[storage.SeriesRef][]byte, []storage.SeriesRef) { + return map[storage.SeriesRef][]byte{}, ids +} + +// BucketStoreOption are functions that configure BucketStore. +type BucketStoreOption func(s *BucketStore) + +// WithLogger sets the BucketStore logger to the one you pass. +func WithLogger(logger log.Logger) BucketStoreOption { + return func(s *BucketStore) { + s.logger = logger + } +} + +// WithRegistry sets a registry that BucketStore uses to register metrics with. +func WithRegistry(reg prometheus.Registerer) BucketStoreOption { + return func(s *BucketStore) { + s.reg = reg + } +} + +// WithIndexCache sets a indexCache to use instead of a noopCache. +func WithIndexCache(cache storecache.IndexCache) BucketStoreOption { + return func(s *BucketStore) { + s.indexCache = cache + } +} + +// WithQueryGate sets a queryGate to use instead of a noopGate. +func WithQueryGate(queryGate gate.Gate) BucketStoreOption { + return func(s *BucketStore) { + s.queryGate = queryGate + } +} + +// WithChunkPool sets a pool.Bytes to use for chunks. +func WithChunkPool(chunkPool pool.Bytes) BucketStoreOption { + return func(s *BucketStore) { + s.chunkPool = chunkPool + } +} + +// WithFilterConfig sets a filter which Store uses for filtering metrics based on time. +func WithFilterConfig(filter *FilterConfig) BucketStoreOption { + return func(s *BucketStore) { + s.filterConfig = filter + } +} + +// WithDebugLogging enables debug logging. +func WithDebugLogging() BucketStoreOption { + return func(s *BucketStore) { + s.debugLogging = true + } +} + +func WithChunkHashCalculation(enableChunkHashCalculation bool) BucketStoreOption { + return func(s *BucketStore) { + s.enableChunkHashCalculation = enableChunkHashCalculation + } +} + +func WithSeriesBatchSize(seriesBatchSize int) BucketStoreOption { + return func(s *BucketStore) { + s.seriesBatchSize = seriesBatchSize + } +} + +func WithBlockEstimatedMaxSeriesFunc(f BlockEstimator) BucketStoreOption { + return func(s *BucketStore) { + s.blockEstimatedMaxSeriesFunc = f + } +} + +func WithBlockEstimatedMaxChunkFunc(f BlockEstimator) BucketStoreOption { + return func(s *BucketStore) { + s.blockEstimatedMaxChunkFunc = f + } +} + +// WithLazyExpandedPostings enables lazy expanded postings. +func WithLazyExpandedPostings(enabled bool) BucketStoreOption { + return func(s *BucketStore) { + s.enabledLazyExpandedPostings = enabled + } +} + +// WithDontResort disables series resorting in Store Gateway. +func WithDontResort(true bool) BucketStoreOption { + return func(s *BucketStore) { + if true { + s.sortingStrategy = sortingStrategyNone + } + } +} + +// NewBucketStore creates a new bucket backed store that implements the store API against +// an object store bucket. It is optimized to work against high latency backends. +func NewBucketStore( + bkt objstore.InstrumentedBucketReader, + fetcher block.MetadataFetcher, + dir string, + chunksLimiterFactory ChunksLimiterFactory, + seriesLimiterFactory SeriesLimiterFactory, + bytesLimiterFactory BytesLimiterFactory, + partitioner Partitioner, + blockSyncConcurrency int, + enableCompatibilityLabel bool, + postingOffsetsInMemSampling int, + enableSeriesResponseHints bool, // TODO(pracucci) Thanos 0.12 and below doesn't gracefully handle new fields in SeriesResponse. Drop this flag and always enable hints once we can drop backward compatibility. + loadIndex bool, + lazyIndexReaderEnabled bool, + lazyIndexReaderIdleTimeout time.Duration, + options ...BucketStoreOption, +) (*BucketStore, error) { + s := &BucketStore{ + logger: log.NewNopLogger(), + bkt: bkt, + fetcher: fetcher, + dir: dir, + indexCache: noopCache{}, + buffers: sync.Pool{New: func() interface{} { + b := make([]byte, 0, initialBufSize) + return &b + }}, + chunkPool: pool.NoopBytes{}, + blocks: map[ulid.ULID]*bucketBlock{}, + blockSets: map[uint64]*bucketBlockSet{}, + blockSyncConcurrency: blockSyncConcurrency, + queryGate: gate.NewNoop(), + chunksLimiterFactory: chunksLimiterFactory, + seriesLimiterFactory: seriesLimiterFactory, + bytesLimiterFactory: bytesLimiterFactory, + loadIndex: loadIndex, + partitioner: partitioner, + enableCompatibilityLabel: enableCompatibilityLabel, + postingOffsetsInMemSampling: postingOffsetsInMemSampling, + enableSeriesResponseHints: enableSeriesResponseHints, + enableChunkHashCalculation: enableChunkHashCalculation, + seriesBatchSize: SeriesBatchSize, + sortingStrategy: sortingStrategyStore, + } + + for _, option := range options { + option(s) + } + + // Depend on the options + indexReaderPoolMetrics := indexheader.NewReaderPoolMetrics(extprom.WrapRegistererWithPrefix("thanos_bucket_store_", s.reg)) + s.indexReaderPool = indexheader.NewReaderPool(s.logger, lazyIndexReaderEnabled, lazyIndexReaderIdleTimeout, indexReaderPoolMetrics) + s.metrics = newBucketStoreMetrics(s.reg) // TODO(metalmatze): Might be possible via Option too + + if err := s.validate(); err != nil { + return nil, errors.Wrap(err, "validate config") + } + + if dir == "" { + return s, nil + } + + if err := os.MkdirAll(dir, 0750); err != nil { + return nil, errors.Wrap(err, "create dir") + } + + return s, nil +} + +// Close the store. +func (s *BucketStore) Close() (err error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + for _, b := range s.blocks { + runutil.CloseWithErrCapture(&err, b, "closing Bucket Block") + } + + s.indexReaderPool.Close() + return err +} + +// SyncBlocks synchronizes the stores state with the Bucket bucket. +// It will reuse disk space as persistent cache based on s.dir param. +func (s *BucketStore) SyncBlocks(ctx context.Context) error { + metas, _, metaFetchErr := s.fetcher.Fetch(ctx) + // For partial view allow adding new blocks at least. + if metaFetchErr != nil && metas == nil { + return metaFetchErr + } + + var wg sync.WaitGroup + blockc := make(chan *metadata.Meta) + + for i := 0; i < s.blockSyncConcurrency; i++ { + wg.Add(1) + go func() { + for meta := range blockc { + if err := s.addBlock(ctx, meta); err != nil { + continue + } + } + wg.Done() + }() + } + + for id, meta := range metas { + if b := s.getBlock(id); b != nil { + continue + } + select { + case <-ctx.Done(): + case blockc <- meta: + } + } + + close(blockc) + wg.Wait() + + if metaFetchErr != nil { + return metaFetchErr + } + + // Drop all blocks that are no longer present in the bucket. + for id := range s.blocks { + if _, ok := metas[id]; ok { + continue + } + if err := s.removeBlock(id); err != nil { + level.Warn(s.logger).Log("msg", "drop of outdated block failed", "block", id, "err", err) + s.metrics.blockDropFailures.Inc() + } + level.Info(s.logger).Log("msg", "dropped outdated block", "block", id) + s.metrics.blockDrops.Inc() + } + + // Sync advertise labels. + var storeLabels labels.Labels + s.mtx.Lock() + s.advLabelSets = make([]labelpb.ZLabelSet, 0, len(s.advLabelSets)) + for _, bs := range s.blockSets { + storeLabels = storeLabels[:0] + s.advLabelSets = append(s.advLabelSets, labelpb.ZLabelSet{Labels: labelpb.ZLabelsFromPromLabels(append(storeLabels, bs.labels...))}) + } + sort.Slice(s.advLabelSets, func(i, j int) bool { + return strings.Compare(s.advLabelSets[i].String(), s.advLabelSets[j].String()) < 0 + }) + s.mtx.Unlock() + return nil +} + +// InitialSync perform blocking sync with extra step at the end to delete locally saved blocks that are no longer +// present in the bucket. The mismatch of these can only happen between restarts, so we can do that only once per startup. +func (s *BucketStore) InitialSync(ctx context.Context) error { + if err := s.SyncBlocks(ctx); err != nil { + return errors.Wrap(err, "sync block") + } + + if s.dir == "" { + return nil + } + + fis, err := os.ReadDir(s.dir) + if err != nil { + return errors.Wrap(err, "read dir") + } + names := make([]string, 0, len(fis)) + for _, fi := range fis { + names = append(names, fi.Name()) + } + for _, n := range names { + id, ok := block.IsBlockDir(n) + if !ok { + continue + } + if b := s.getBlock(id); b != nil { + continue + } + + // No such block loaded, remove the local dir. + if err := os.RemoveAll(path.Join(s.dir, id.String())); err != nil { + level.Warn(s.logger).Log("msg", "failed to remove block which is not needed", "err", err) + } + } + + return nil +} + +func (s *BucketStore) getBlock(id ulid.ULID) *bucketBlock { + s.mtx.RLock() + defer s.mtx.RUnlock() + return s.blocks[id] +} + +func (s *BucketStore) addBlock(ctx context.Context, meta *metadata.Meta) (err error) { + var dir string + if s.dir != "" { + dir = filepath.Join(s.dir, meta.ULID.String()) + } + start := time.Now() + + level.Debug(s.logger).Log("msg", "loading new block", "id", meta.ULID) + defer func() { + if err != nil { + s.metrics.blockLoadFailures.Inc() + if dir != "" { + if err2 := os.RemoveAll(dir); err2 != nil { + level.Warn(s.logger).Log("msg", "failed to remove block we cannot load", "err", err2) + } + } + level.Warn(s.logger).Log("msg", "loading block failed", "elapsed", time.Since(start), "id", meta.ULID, "err", err) + } else { + level.Info(s.logger).Log("msg", "loaded new block", "elapsed", time.Since(start), "id", meta.ULID) + s.metrics.blockLoadDuration.Observe(time.Since(start).Seconds()) + } + }() + s.metrics.blockLoads.Inc() + + lset := labels.FromMap(meta.Thanos.Labels) + h := lset.Hash() + + var indexHeaderReader indexheader.Reader = nil + + if s.loadIndex { + indexHeaderReader, err = s.indexReaderPool.NewBinaryReader( + ctx, + s.logger, + s.bkt, + s.dir, + meta.ULID, + s.postingOffsetsInMemSampling, + ) + if err != nil { + return errors.Wrap(err, "create index header reader") + } + } + + defer func() { + if err != nil { + runutil.CloseWithErrCapture(&err, indexHeaderReader, "index-header") + } + }() + + b, err := newBucketBlock( + ctx, + log.With(s.logger, "block", meta.ULID), + s.metrics, + meta, + s.bkt, + dir, + s.indexCache, + s.chunkPool, + indexHeaderReader, + s.partitioner, + s.blockEstimatedMaxSeriesFunc, + s.blockEstimatedMaxChunkFunc, + ) + if err != nil { + return errors.Wrap(err, "new bucket block") + } + defer func() { + if err != nil { + runutil.CloseWithErrCapture(&err, b, "index-header") + } + }() + + s.mtx.Lock() + defer s.mtx.Unlock() + + sort.Sort(lset) + + set, ok := s.blockSets[h] + if !ok { + set = newBucketBlockSet(lset) + s.blockSets[h] = set + } + + if err = set.add(b); err != nil { + return errors.Wrap(err, "add block to set") + } + s.blocks[b.meta.ULID] = b + + s.metrics.blocksLoaded.Inc() + s.metrics.lastLoadedBlock.SetToCurrentTime() + return nil +} + +func (s *BucketStore) removeBlock(id ulid.ULID) error { + s.mtx.Lock() + b, ok := s.blocks[id] + if ok { + lset := labels.FromMap(b.meta.Thanos.Labels) + s.blockSets[lset.Hash()].remove(id) + delete(s.blocks, id) + } + s.mtx.Unlock() + + if !ok { + return nil + } + + s.metrics.blocksLoaded.Dec() + if err := b.Close(); err != nil { + return errors.Wrap(err, "close block") + } + + if b.dir == "" { + return nil + } + + return os.RemoveAll(b.dir) +} + +// TimeRange returns the minimum and maximum timestamp of data available in the store. +func (s *BucketStore) TimeRange() (mint, maxt int64) { + s.mtx.RLock() + defer s.mtx.RUnlock() + + mint = math.MaxInt64 + maxt = math.MinInt64 + + for _, b := range s.blocks { + if b.meta.MinTime < mint { + mint = b.meta.MinTime + } + if b.meta.MaxTime > maxt { + maxt = b.meta.MaxTime + } + } + + mint = s.limitMinTime(mint) + maxt = s.limitMaxTime(maxt) + + return mint, maxt +} + +// TSDBInfos returns a list of infopb.TSDBInfos for blocks in the bucket store. +func (s *BucketStore) TSDBInfos() []infopb.TSDBInfo { + s.mtx.RLock() + defer s.mtx.RUnlock() + + infos := make([]infopb.TSDBInfo, 0, len(s.blocks)) + for _, b := range s.blocks { + infos = append(infos, infopb.TSDBInfo{ + Labels: labelpb.ZLabelSet{ + Labels: labelpb.ZLabelsFromPromLabels(labels.FromMap(b.meta.Thanos.Labels)), + }, + MinTime: b.meta.MinTime, + MaxTime: b.meta.MaxTime, + }) + } + + return infos +} + +func (s *BucketStore) LabelSet() []labelpb.ZLabelSet { + s.mtx.RLock() + labelSets := s.advLabelSets + s.mtx.RUnlock() + + if s.enableCompatibilityLabel && len(labelSets) > 0 { + labelSets = append(labelSets, labelpb.ZLabelSet{Labels: []labelpb.ZLabel{{Name: CompatibilityTypeLabelName, Value: "store"}}}) + } + + return labelSets +} + +func (s *BucketStore) limitMinTime(mint int64) int64 { + if s.filterConfig == nil { + return mint + } + + filterMinTime := s.filterConfig.MinTime.PrometheusTimestamp() + + if mint < filterMinTime { + return filterMinTime + } + + return mint +} + +func (s *BucketStore) limitMaxTime(maxt int64) int64 { + if s.filterConfig == nil { + return maxt + } + + filterMaxTime := s.filterConfig.MaxTime.PrometheusTimestamp() + + if maxt > filterMaxTime { + maxt = filterMaxTime + } + + return maxt +} + +func populateChunk(out *typespb.AggrChunk, in chunkenc.Chunk, aggrs []storepb.Aggr, save func([]byte) ([]byte, error), calculateChecksum bool) error { + hasher := hashPool.Get().(hash.Hash64) + defer hashPool.Put(hasher) + + if in.Encoding() == chunkenc.EncXOR || in.Encoding() == chunkenc.EncHistogram { + b, err := save(in.Bytes()) + if err != nil { + return err + } + out.Raw = &typespb.Chunk{ + Data: b, + Type: typespb.Chunk_Encoding(in.Encoding() - 1), + Hash: hashChunk(hasher, b, calculateChecksum), + } + return nil + } + + if in.Encoding() != downsample.ChunkEncAggr { + return errors.Errorf("unsupported chunk encoding %d", in.Encoding()) + } + + ac := downsample.AggrChunk(in.Bytes()) + + for _, at := range aggrs { + switch at { + case storepb.COUNT: + x, err := ac.Get(downsample.AggrCount) + if err != nil { + return errors.Errorf("aggregate %s does not exist", downsample.AggrCount) + } + b, err := save(x.Bytes()) + if err != nil { + return err + } + out.Count = &typespb.Chunk{Type: typespb.XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + case storepb.SUM: + x, err := ac.Get(downsample.AggrSum) + if err != nil { + return errors.Errorf("aggregate %s does not exist", downsample.AggrSum) + } + b, err := save(x.Bytes()) + if err != nil { + return err + } + out.Sum = &typespb.Chunk{Type: typespb.XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + case storepb.MIN: + x, err := ac.Get(downsample.AggrMin) + if err != nil { + return errors.Errorf("aggregate %s does not exist", downsample.AggrMin) + } + b, err := save(x.Bytes()) + if err != nil { + return err + } + out.Min = &typespb.Chunk{Type: typespb.XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + case storepb.MAX: + x, err := ac.Get(downsample.AggrMax) + if err != nil { + return errors.Errorf("aggregate %s does not exist", downsample.AggrMax) + } + b, err := save(x.Bytes()) + if err != nil { + return err + } + out.Max = &typespb.Chunk{Type: typespb.XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + case storepb.COUNTER: + x, err := ac.Get(downsample.AggrCounter) + if err != nil { + return errors.Errorf("aggregate %s does not exist", downsample.AggrCounter) + } + b, err := save(x.Bytes()) + if err != nil { + return err + } + out.Counter = &typespb.Chunk{Type: typespb.XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + } + } + return nil +} + +func hashChunk(hasher hash.Hash64, b []byte, doHash bool) uint64 { + if !doHash { + return 0 + } + hasher.Reset() + // Write never returns an error on the hasher implementation + _, _ = hasher.Write(b) + return hasher.Sum64() +} + +// debugFoundBlockSetOverview logs on debug level what exactly blocks we used for query in terms of +// labels and resolution. This is important because we allow mixed resolution results, so it is quite crucial +// to be aware what exactly resolution we see on query. +// TODO(bplotka): Consider adding resolution label to all results to propagate that info to UI and Query API. +func debugFoundBlockSetOverview(logger log.Logger, mint, maxt, maxResolutionMillis int64, lset labels.Labels, bs []*bucketBlock) { + if len(bs) == 0 { + level.Debug(logger).Log("msg", "No block found", "mint", mint, "maxt", maxt, "lset", lset.String()) + return + } + + var ( + parts []string + currRes = int64(-1) + currMin, currMax int64 + ) + for _, b := range bs { + if currRes == b.meta.Thanos.Downsample.Resolution { + currMax = b.meta.MaxTime + continue + } + + if currRes != -1 { + parts = append(parts, fmt.Sprintf("Range: %d-%d Resolution: %d", currMin, currMax, currRes)) + } + + currRes = b.meta.Thanos.Downsample.Resolution + currMin = b.meta.MinTime + currMax = b.meta.MaxTime + } + + parts = append(parts, fmt.Sprintf("Range: %d-%d Resolution: %d", currMin, currMax, currRes)) + + level.Debug(logger).Log("msg", "Blocks source resolutions", "blocks", len(bs), "Maximum Resolution", maxResolutionMillis, "mint", mint, "maxt", maxt, "lset", lset.String(), "spans", strings.Join(parts, "\n")) +} + +// Series implements the storepb.StoreServer interface. +func (s *BucketStore) Series(req *storepb.SeriesRequest, seriesSrv storepb.Store_SeriesServer) (err error) { + srv := newFlushableServer(seriesSrv, sortingStrategyNone) + + if s.queryGate != nil { + tracing.DoInSpan(srv.Context(), "store_query_gate_ismyturn", func(ctx context.Context) { + err = s.queryGate.Start(srv.Context()) + }) + if err != nil { + return errors.Wrapf(err, "failed to wait for turn") + } + + defer s.queryGate.Done() + } + + tenant, _ := tenancy.GetTenantFromGRPCMetadata(srv.Context()) + + matchers, err := typespb.MatchersToPromMatchers(req.Matchers...) + if err != nil { + return status.Error(codes.InvalidArgument, err.Error()) + } + req.MinTime = s.limitMinTime(req.MinTime) + req.MaxTime = s.limitMaxTime(req.MaxTime) + + var ( + bytesLimiter = s.bytesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("bytes", tenant)) + ctx = srv.Context() + stats = &queryStats{} + respSets []respSet + mtx sync.Mutex + g, gctx = errgroup.WithContext(ctx) + resHints = &hintspb.SeriesResponseHints{} + reqBlockMatchers []*labels.Matcher + chunksLimiter = s.chunksLimiterFactory(s.metrics.queriesDropped.WithLabelValues("chunks", tenant)) + seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series", tenant)) + + queryStatsEnabled = false + ) + + if req.Hints != nil { + reqHints := &hintspb.SeriesRequestHints{} + if err := types.UnmarshalAny(req.Hints, reqHints); err != nil { + return status.Error(codes.InvalidArgument, errors.Wrap(err, "unmarshal series request hints").Error()) + } + queryStatsEnabled = reqHints.EnableQueryStats + + reqBlockMatchers, err = typespb.MatchersToPromMatchers(reqHints.BlockMatchers...) + if err != nil { + return status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request hints labels matchers").Error()) + } + } + + var extLsetToRemove map[string]struct{} + if len(req.WithoutReplicaLabels) > 0 { + extLsetToRemove = make(map[string]struct{}) + for _, l := range req.WithoutReplicaLabels { + extLsetToRemove[l] = struct{}{} + } + } + + s.mtx.RLock() + for _, bs := range s.blockSets { + blockMatchers, ok := bs.labelMatchers(matchers...) + if !ok { + continue + } + // Sort matchers to make sure we generate the same cache key + // when fetching expanded postings. + sortedBlockMatchers := newSortedMatchers(blockMatchers) + + blocks := bs.getFor(req.MinTime, req.MaxTime, req.MaxResolutionWindow, reqBlockMatchers) + + if s.debugLogging { + debugFoundBlockSetOverview(s.logger, req.MinTime, req.MaxTime, req.MaxResolutionWindow, bs.labels, blocks) + } + + for _, b := range blocks { + blk := b + gctx := gctx + + if s.enableSeriesResponseHints { + // Keep track of queried blocks. + resHints.AddQueriedBlock(blk.meta.ULID) + } + + shardMatcher := req.ShardInfo.Matcher(&s.buffers) + + blockClient := newBlockSeriesClient( + srv.Context(), + s.logger, + blk, + req, + chunksLimiter, + bytesLimiter, + sortedBlockMatchers, + shardMatcher, + s.enableChunkHashCalculation, + s.seriesBatchSize, + s.metrics.seriesFetchDurationSum, + s.metrics.chunkFetchDuration, + s.metrics.chunkFetchDurationSum, + extLsetToRemove, + s.enabledLazyExpandedPostings, + s.metrics.lazyExpandedPostingsCount, + s.metrics.lazyExpandedPostingSizeBytes, + s.metrics.lazyExpandedPostingSeriesOverfetchedSizeBytes, + tenant, + ) + + defer blockClient.Close() + + g.Go(func() error { + + span, _ := tracing.StartSpan(gctx, "bucket_store_block_series", tracing.Tags{ + "block.id": blk.meta.ULID, + "block.mint": blk.meta.MinTime, + "block.maxt": blk.meta.MaxTime, + "block.resolution": blk.meta.Thanos.Downsample.Resolution, + }) + + onClose := func() { + mtx.Lock() + stats = blockClient.MergeStats(stats) + mtx.Unlock() + } + + if err := blockClient.ExpandPostings( + sortedBlockMatchers, + seriesLimiter, + ); err != nil { + onClose() + span.Finish() + return errors.Wrapf(err, "fetch postings for block %s", blk.meta.ULID) + } + + var resp respSet + if s.sortingStrategy == sortingStrategyStore { + resp = newEagerRespSet( + srv.Context(), + span, + 10*time.Minute, + blk.meta.ULID.String(), + []labels.Labels{blk.extLset}, + onClose, + blockClient, + shardMatcher, + false, + s.metrics.emptyPostingCount.WithLabelValues(tenant), + nil, + ) + } else { + resp = newLazyRespSet( + srv.Context(), + span, + 10*time.Minute, + blk.meta.ULID.String(), + []labels.Labels{blk.extLset}, + onClose, + blockClient, + shardMatcher, + false, + s.metrics.emptyPostingCount.WithLabelValues(tenant), + ) + } + + mtx.Lock() + respSets = append(respSets, resp) + mtx.Unlock() + + return nil + }) + } + } + + s.mtx.RUnlock() + + defer func() { + s.metrics.seriesDataTouched.WithLabelValues("postings", tenant).Observe(float64(stats.postingsTouched)) + s.metrics.seriesDataFetched.WithLabelValues("postings", tenant).Observe(float64(stats.postingsFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("postings", tenant).Observe(float64(stats.PostingsTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("postings", tenant).Observe(float64(stats.PostingsFetchedSizeSum)) + s.metrics.seriesDataTouched.WithLabelValues("series", tenant).Observe(float64(stats.seriesTouched)) + s.metrics.seriesDataFetched.WithLabelValues("series", tenant).Observe(float64(stats.seriesFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("series", tenant).Observe(float64(stats.SeriesTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("series", tenant).Observe(float64(stats.SeriesFetchedSizeSum)) + s.metrics.seriesDataTouched.WithLabelValues("chunks", tenant).Observe(float64(stats.chunksTouched)) + s.metrics.seriesDataFetched.WithLabelValues("chunks", tenant).Observe(float64(stats.chunksFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("chunks", tenant).Observe(float64(stats.ChunksTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("chunks", tenant).Observe(float64(stats.ChunksFetchedSizeSum)) + s.metrics.resultSeriesCount.WithLabelValues(tenant).Observe(float64(stats.mergedSeriesCount)) + s.metrics.cachedPostingsCompressions.WithLabelValues(labelEncode, tenant).Add(float64(stats.cachedPostingsCompressions)) + s.metrics.cachedPostingsCompressions.WithLabelValues(labelDecode, tenant).Add(float64(stats.cachedPostingsDecompressions)) + s.metrics.cachedPostingsCompressionErrors.WithLabelValues(labelEncode, tenant).Add(float64(stats.cachedPostingsCompressionErrors)) + s.metrics.cachedPostingsCompressionErrors.WithLabelValues(labelDecode, tenant).Add(float64(stats.cachedPostingsDecompressionErrors)) + s.metrics.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelEncode, tenant).Add(stats.CachedPostingsCompressionTimeSum.Seconds()) + s.metrics.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelDecode, tenant).Add(stats.CachedPostingsDecompressionTimeSum.Seconds()) + s.metrics.cachedPostingsOriginalSizeBytes.WithLabelValues(tenant).Add(float64(stats.CachedPostingsOriginalSizeSum)) + s.metrics.cachedPostingsCompressedSizeBytes.WithLabelValues(tenant).Add(float64(stats.CachedPostingsCompressedSizeSum)) + s.metrics.postingsSizeBytes.WithLabelValues(tenant).Observe(float64(int(stats.PostingsFetchedSizeSum) + int(stats.PostingsTouchedSizeSum))) + + level.Debug(s.logger).Log("msg", "stats query processed", + "request", req, + "tenant", tenant, + "stats", fmt.Sprintf("%+v", stats), "err", err) + }() + + // Concurrently get data from all blocks. + { + begin := time.Now() + tracing.DoInSpan(ctx, "bucket_store_preload_all", func(_ context.Context) { + err = g.Wait() + }) + if err != nil { + code := codes.Aborted + if s, ok := status.FromError(errors.Cause(err)); ok { + code = s.Code() + } + return status.Error(code, err.Error()) + } + stats.blocksQueried = len(respSets) + stats.GetAllDuration = time.Since(begin) + s.metrics.seriesGetAllDuration.WithLabelValues(tenant).Observe(stats.GetAllDuration.Seconds()) + s.metrics.seriesBlocksQueried.WithLabelValues(tenant).Observe(float64(stats.blocksQueried)) + } + + // Merge the sub-results from each selected block. + tracing.DoInSpan(ctx, "bucket_store_merge_all", func(ctx context.Context) { + defer func() { + for _, resp := range respSets { + resp.Close() + } + }() + begin := time.Now() + set := NewDedupResponseHeap(NewProxyResponseHeap(respSets...)) + for set.Next() { + at := set.At() + warn := at.GetWarning() + if warn != "" { + // TODO(fpetkovski): Consider deprecating string based warnings in favor of a + // separate protobuf message containing the grpc code and + // a human readable error message. + err = status.Error(storepb.GRPCCodeFromWarn(warn), at.GetWarning()) + return + } + + series := at.GetSeries() + if series != nil { + stats.mergedSeriesCount++ + if !req.SkipChunks { + stats.mergedChunksCount += len(series.Chunks) + s.metrics.chunkSizeBytes.WithLabelValues(tenant).Observe(float64(chunksSize(series.Chunks))) + } + } + if err = srv.Send(at); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "send series response").Error()) + return + } + } + stats.MergeDuration = time.Since(begin) + s.metrics.seriesMergeDuration.WithLabelValues(tenant).Observe(stats.MergeDuration.Seconds()) + + err = nil + }) + if err != nil { + return err + } + + if s.enableSeriesResponseHints { + var anyHints *types.Any + + if queryStatsEnabled { + resHints.QueryStats = stats.toHints() + } + if anyHints, err = types.MarshalAny(resHints); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "marshal series response hints").Error()) + return + } + + if err = srv.Send(storepb.NewHintsSeriesResponse(anyHints)); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "send series response hints").Error()) + return + } + } + + if err != nil { + return err + } + return srv.Flush() +} + +func (s *BucketStore) Select(req *storepb.SelectRequest, srv storepb.Store_SelectServer) (err error) { + if s.queryGate != nil { + tracing.DoInSpan(srv.Context(), "store_query_gate_ismyturn", func(ctx context.Context) { + err = s.queryGate.Start(srv.Context()) + }) + if err != nil { + return errors.Wrapf(err, "failed to wait for turn") + } + + defer s.queryGate.Done() + } + + tenant, _ := tenancy.GetTenantFromGRPCMetadata(srv.Context()) + + matchers, err := typespb.MatchersToPromMatchers(req.Matchers...) + if err != nil { + return status.Error(codes.InvalidArgument, err.Error()) + } + req.MinTime = s.limitMinTime(req.MinTime) + req.MaxTime = s.limitMaxTime(req.MaxTime) + + var ( + bytesLimiter = s.bytesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("bytes", tenant)) + stats = &queryStats{} + mtx sync.Mutex + resHints = &hintspb.SeriesResponseHints{} + reqBlockMatchers []*labels.Matcher + seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series", tenant)) + ) + + if req.Hints != nil { + reqHints := &hintspb.SeriesRequestHints{} + if err := types.UnmarshalAny(req.Hints, reqHints); err != nil { + return status.Error(codes.InvalidArgument, errors.Wrap(err, "unmarshal series request hints").Error()) + } + + reqBlockMatchers, err = typespb.MatchersToPromMatchers(reqHints.BlockMatchers...) + if err != nil { + return status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request hints labels matchers").Error()) + } + } + + var extLsetToRemove map[string]struct{} + if len(req.WithoutReplicaLabels) > 0 { + extLsetToRemove = make(map[string]struct{}) + for _, l := range req.WithoutReplicaLabels { + extLsetToRemove[l] = struct{}{} + } + } + + s.mtx.RLock() + for _, bs := range s.blockSets { + blockMatchers, ok := bs.labelMatchers(matchers...) + if !ok { + continue + } + // Sort matchers to make sure we generate the same cache key + // when fetching expanded postings. + sortedBlockMatchers := newSortedMatchers(blockMatchers) + + blocks := bs.getFor(req.MinTime, req.MaxTime, req.MaxResolutionWindow, reqBlockMatchers) + + if s.debugLogging { + debugFoundBlockSetOverview(s.logger, req.MinTime, req.MaxTime, req.MaxResolutionWindow, bs.labels, blocks) + } + + // There should be only one block. + if len(blocks) > 1 { + return status.Error(codes.Internal, errors.Wrap(err, "too many blocks queried").Error()) + } + + stats.blocksQueried = len(blocks) + + for _, b := range blocks { + blk := b + + if s.enableSeriesResponseHints { + // Keep track of queried blocks. + resHints.AddQueriedBlock(blk.meta.ULID) + } + + shardMatcher := req.ShardInfo.Matcher(&s.buffers) + + blockClient := newBlockIndexClient( + srv.Context(), + s.logger, + blk, + blockIndexRequest{ + maxTime: req.MaxTime, + minTime: req.MinTime, + skipChunks: req.SkipChunks, + }, + bytesLimiter, + sortedBlockMatchers, + shardMatcher, + s.seriesBatchSize, + s.metrics.seriesFetchDurationSum, + extLsetToRemove, + s.enabledLazyExpandedPostings, + s.metrics.lazyExpandedPostingsCount, + s.metrics.lazyExpandedPostingSizeBytes, + s.metrics.lazyExpandedPostingSeriesOverfetchedSizeBytes, + tenant, + ) + + defer blockClient.Close() + + span, _ := tracing.StartSpan(blockClient.ctx, "bucket_store_block_series", tracing.Tags{ + "block.id": blk.meta.ULID, + "block.mint": blk.meta.MinTime, + "block.maxt": blk.meta.MaxTime, + "block.resolution": blk.meta.Thanos.Downsample.Resolution, + }) + + onClose := func() { + mtx.Lock() + stats = blockClient.MergeStats(stats) + mtx.Unlock() + } + + if err := blockClient.ExpandPostings( + sortedBlockMatchers, + seriesLimiter, + ); err != nil { + onClose() + span.Finish() + return errors.Wrapf(err, "fetch postings for block %s", blk.meta.ULID) + } + + for blockClient.hasNext() { + entries, err := blockClient.nextBatch(tenant) + if err != nil { + return err + } + + for _, e := range entries { + srv.Send(storepb.NewSelectResponse(&typespb.SelectedSeries{ + BlockId: b.meta.ULID.String(), + Labels: labelpb.ZLabelsFromPromLabels(e.lset), + Chunks: e.meta, + })) + } + } + + } + } + + s.mtx.RUnlock() + + defer func() { + s.metrics.seriesDataTouched.WithLabelValues("postings", tenant).Observe(float64(stats.postingsTouched)) + s.metrics.seriesDataFetched.WithLabelValues("postings", tenant).Observe(float64(stats.postingsFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("postings", tenant).Observe(float64(stats.PostingsTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("postings", tenant).Observe(float64(stats.PostingsFetchedSizeSum)) + s.metrics.seriesDataTouched.WithLabelValues("series", tenant).Observe(float64(stats.seriesTouched)) + s.metrics.seriesDataFetched.WithLabelValues("series", tenant).Observe(float64(stats.seriesFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("series", tenant).Observe(float64(stats.SeriesTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("series", tenant).Observe(float64(stats.SeriesFetchedSizeSum)) + s.metrics.seriesDataTouched.WithLabelValues("chunks", tenant).Observe(float64(stats.chunksTouched)) + s.metrics.seriesDataFetched.WithLabelValues("chunks", tenant).Observe(float64(stats.chunksFetched)) + s.metrics.seriesDataSizeTouched.WithLabelValues("chunks", tenant).Observe(float64(stats.ChunksTouchedSizeSum)) + s.metrics.seriesDataSizeFetched.WithLabelValues("chunks", tenant).Observe(float64(stats.ChunksFetchedSizeSum)) + s.metrics.resultSeriesCount.WithLabelValues(tenant).Observe(float64(stats.mergedSeriesCount)) + s.metrics.cachedPostingsCompressions.WithLabelValues(labelEncode, tenant).Add(float64(stats.cachedPostingsCompressions)) + s.metrics.cachedPostingsCompressions.WithLabelValues(labelDecode, tenant).Add(float64(stats.cachedPostingsDecompressions)) + s.metrics.cachedPostingsCompressionErrors.WithLabelValues(labelEncode, tenant).Add(float64(stats.cachedPostingsCompressionErrors)) + s.metrics.cachedPostingsCompressionErrors.WithLabelValues(labelDecode, tenant).Add(float64(stats.cachedPostingsDecompressionErrors)) + s.metrics.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelEncode, tenant).Add(stats.CachedPostingsCompressionTimeSum.Seconds()) + s.metrics.cachedPostingsCompressionTimeSeconds.WithLabelValues(labelDecode, tenant).Add(stats.CachedPostingsDecompressionTimeSum.Seconds()) + s.metrics.cachedPostingsOriginalSizeBytes.WithLabelValues(tenant).Add(float64(stats.CachedPostingsOriginalSizeSum)) + s.metrics.cachedPostingsCompressedSizeBytes.WithLabelValues(tenant).Add(float64(stats.CachedPostingsCompressedSizeSum)) + s.metrics.postingsSizeBytes.WithLabelValues(tenant).Observe(float64(int(stats.PostingsFetchedSizeSum) + int(stats.PostingsTouchedSizeSum))) + + level.Debug(s.logger).Log("msg", "stats query processed", + "request", req, + "tenant", tenant, + "stats", fmt.Sprintf("%+v", stats), "err", err) + }() + return nil +} + +func (s *BucketStore) Chunks(srv storepb.ChunkStore_ChunksServer) (err error) { + var block *bucketBlock + var client *blockChunkClient + tenant, _ := tenancy.GetTenantFromGRPCMetadata(srv.Context()) + var bytesLimiter = s.bytesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("bytes", tenant)) + var chunksLimiter = s.chunksLimiterFactory(s.metrics.queriesDropped.WithLabelValues("chunks", tenant)) + + entries := []seriesEntry{} + + for { + in, err := srv.Recv() + if err == io.EOF { + break + } + + if err != nil { + return err + } + + if block == nil { + id, err := ulid.Parse(in.BlockId) + if err != nil { + return err + } + + if block = s.getBlock(id); block == nil { + return errors.New("block not found") + } + } + refs := make([]chunks.ChunkRef, 0, len(in.Chunkref)) + chks := make([]typespb.AggrChunk, 0, len(in.Chunkref)) + + for _, r := range in.Chunkref { + refs = append(refs, chunks.ChunkRef(r)) + chks = append(chks, typespb.AggrChunk{}) + } + + entries = append(entries, seriesEntry{ + refs: refs, + chks: chks, + }) + } + + if len(entries) == 0 { + return + } + + client = newBlockChunkClient( + srv.Context(), + s.logger, + block, + chunksLimiter, + bytesLimiter, + s.enableChunkHashCalculation, + s.metrics.chunkFetchDuration, + s.metrics.chunkFetchDurationSum, + ) + + defer client.Close() + + if err := client.loadChunks(entries); err != nil { + return err + } + + for _, entry := range entries { + chks := make([]*typespb.AggrChunk, 0, len(entry.chks)) + for _, c := range entry.chks { + chks = append(chks, &c) + } + srv.Send(&storepb.ChunksResponse{ + Chunks: chks, + }) + } + + return nil +} + +func chunksSize(chks []typespb.AggrChunk) (size int) { + for _, chk := range chks { + size += chk.Size() // This gets the encoded proto size. + } + return size +} + +// LabelNames implements the storepb.StoreServer interface. +func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { + reqSeriesMatchers, err := typespb.MatchersToPromMatchers(req.Matchers...) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request labels matchers").Error()) + } + + tenant, _ := tenancy.GetTenantFromGRPCMetadata(ctx) + + resHints := &hintspb.LabelNamesResponseHints{} + + var reqBlockMatchers []*labels.Matcher + if req.Hints != nil { + reqHints := &hintspb.LabelNamesRequestHints{} + err := types.UnmarshalAny(req.Hints, reqHints) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "unmarshal label names request hints").Error()) + } + + reqBlockMatchers, err = typespb.MatchersToPromMatchers(reqHints.BlockMatchers...) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request hints labels matchers").Error()) + } + } + + g, gctx := errgroup.WithContext(ctx) + + s.mtx.RLock() + + var mtx sync.Mutex + var sets [][]string + var seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series", tenant)) + var bytesLimiter = s.bytesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("bytes", tenant)) + + for _, b := range s.blocks { + b := b + gctx := gctx + + if !b.overlapsClosedInterval(req.Start, req.End) { + continue + } + if len(reqBlockMatchers) > 0 && !b.matchRelabelLabels(reqBlockMatchers) { + continue + } + // Filter external labels from matchers. + reqSeriesMatchersNoExtLabels, ok := b.FilterExtLabelsMatchers(reqSeriesMatchers) + if !ok { + continue + } + + sortedReqSeriesMatchersNoExtLabels := newSortedMatchers(reqSeriesMatchersNoExtLabels) + + resHints.AddQueriedBlock(b.meta.ULID) + + indexr := b.indexReader() + + g.Go(func() error { + span, newCtx := tracing.StartSpan(gctx, "bucket_store_block_series", tracing.Tags{ + "block.id": b.meta.ULID, + "block.mint": b.meta.MinTime, + "block.maxt": b.meta.MaxTime, + "block.resolution": b.meta.Thanos.Downsample.Resolution, + }) + defer span.Finish() + defer runutil.CloseWithLogOnErr(s.logger, indexr, "label names") + + var result []string + if len(reqSeriesMatchersNoExtLabels) == 0 { + // Do it via index reader to have pending reader registered correctly. + // LabelNames are already sorted. + res, err := indexr.block.indexHeaderReader.LabelNames() + if err != nil { + return errors.Wrapf(err, "label names for block %s", b.meta.ULID) + } + + // Add a set for the external labels as well. + // We're not adding them directly to refs because there could be duplicates. + // b.extLset is already sorted by label name, no need to sort it again. + extRes := make([]string, 0, len(b.extLset)) + for _, l := range b.extLset { + extRes = append(extRes, l.Name) + } + + result = strutil.MergeSlices(res, extRes) + } else { + seriesReq := &storepb.SeriesRequest{ + MinTime: req.Start, + MaxTime: req.End, + SkipChunks: true, + } + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + reqSeriesMatchersNoExtLabels, + nil, + true, + SeriesBatchSize, + s.metrics.seriesFetchDurationSum, + nil, + nil, + nil, + s.enabledLazyExpandedPostings, + s.metrics.lazyExpandedPostingsCount, + s.metrics.lazyExpandedPostingSizeBytes, + s.metrics.lazyExpandedPostingSeriesOverfetchedSizeBytes, + tenant, + ) + defer blockClient.Close() + + if err := blockClient.ExpandPostings( + sortedReqSeriesMatchersNoExtLabels, + seriesLimiter, + ); err != nil { + return err + } + + // Extract label names from all series. Many label names will be the same, so we need to deduplicate them. + // Note that label names will already include external labels (passed to blockSeries), so we don't need + // to add them again. + labelNames := map[string]struct{}{} + for { + ls, err := blockClient.Recv() + if err == io.EOF { + break + } + if err != nil { + return errors.Wrapf(err, "iterate series for block %s", b.meta.ULID) + } + + if ls.GetWarning() != "" { + return errors.Wrapf(errors.New(ls.GetWarning()), "iterate series for block %s", b.meta.ULID) + } + if ls.GetSeries() == nil { + continue + } + for _, l := range ls.GetSeries().Labels { + labelNames[l.Name] = struct{}{} + } + } + + result = make([]string, 0, len(labelNames)) + for n := range labelNames { + result = append(result, n) + } + sort.Strings(result) + } + + if len(result) > 0 { + mtx.Lock() + sets = append(sets, result) + mtx.Unlock() + } + + return nil + }) + } + + s.mtx.RUnlock() + + if err := g.Wait(); err != nil { + code := codes.Internal + if s, ok := status.FromError(errors.Cause(err)); ok { + code = s.Code() + } + return nil, status.Error(code, err.Error()) + } + + anyHints, err := types.MarshalAny(resHints) + if err != nil { + return nil, status.Error(codes.Unknown, errors.Wrap(err, "marshal label names response hints").Error()) + } + + return &storepb.LabelNamesResponse{ + Names: strutil.MergeSlices(sets...), + Hints: anyHints, + }, nil +} + +func (b *bucketBlock) FilterExtLabelsMatchers(matchers []*labels.Matcher) ([]*labels.Matcher, bool) { + // We filter external labels from matchers so we won't try to match series on them. + var result []*labels.Matcher + for _, m := range matchers { + // Get value of external label from block. + v := b.extLset.Get(m.Name) + // If value is empty string the matcher is a valid one since it's not part of external labels. + if v == "" { + result = append(result, m) + } else if v != "" && v != m.Value { + // If matcher is external label but value is different we don't want to look in block anyway. + return []*labels.Matcher{}, false + } + } + + return result, true +} + +// LabelValues implements the storepb.StoreServer interface. +func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error) { + reqSeriesMatchers, err := typespb.MatchersToPromMatchers(req.Matchers...) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request labels matchers").Error()) + } + + tenant, _ := tenancy.GetTenantFromGRPCMetadata(ctx) + + resHints := &hintspb.LabelValuesResponseHints{} + + g, gctx := errgroup.WithContext(ctx) + + var reqBlockMatchers []*labels.Matcher + if req.Hints != nil { + reqHints := &hintspb.LabelValuesRequestHints{} + err := types.UnmarshalAny(req.Hints, reqHints) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "unmarshal label values request hints").Error()) + } + + reqBlockMatchers, err = typespb.MatchersToPromMatchers(reqHints.BlockMatchers...) + if err != nil { + return nil, status.Error(codes.InvalidArgument, errors.Wrap(err, "translate request hints labels matchers").Error()) + } + } + + s.mtx.RLock() + + var mtx sync.Mutex + var sets [][]string + var seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series", tenant)) + var bytesLimiter = s.bytesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("bytes", tenant)) + + for _, b := range s.blocks { + b := b + + if !b.overlapsClosedInterval(req.Start, req.End) { + continue + } + if len(reqBlockMatchers) > 0 && !b.matchRelabelLabels(reqBlockMatchers) { + continue + } + // Filter external labels from matchers. + reqSeriesMatchersNoExtLabels, ok := b.FilterExtLabelsMatchers(reqSeriesMatchers) + if !ok { + continue + } + + // If we have series matchers, add != "" matcher, to only select series that have given label name. + if len(reqSeriesMatchersNoExtLabels) > 0 { + m, err := labels.NewMatcher(labels.MatchNotEqual, req.Label, "") + if err != nil { + return nil, status.Error(codes.InvalidArgument, err.Error()) + } + + reqSeriesMatchersNoExtLabels = append(reqSeriesMatchersNoExtLabels, m) + } + + sortedReqSeriesMatchersNoExtLabels := newSortedMatchers(reqSeriesMatchersNoExtLabels) + + resHints.AddQueriedBlock(b.meta.ULID) + + indexr := b.indexReader() + g.Go(func() error { + span, newCtx := tracing.StartSpan(gctx, "bucket_store_block_series", tracing.Tags{ + "block.id": b.meta.ULID, + "block.mint": b.meta.MinTime, + "block.maxt": b.meta.MaxTime, + "block.resolution": b.meta.Thanos.Downsample.Resolution, + }) + defer span.Finish() + defer runutil.CloseWithLogOnErr(s.logger, indexr, "label values") + + var result []string + if len(reqSeriesMatchersNoExtLabels) == 0 { + // Do it via index reader to have pending reader registered correctly. + res, err := indexr.block.indexHeaderReader.LabelValues(req.Label) + if err != nil { + return errors.Wrapf(err, "index header label values for block %s", b.meta.ULID) + } + + // Add the external label value as well. + if extLabelValue := b.extLset.Get(req.Label); extLabelValue != "" { + res = strutil.MergeSlices(res, []string{extLabelValue}) + } + result = res + } else { + seriesReq := &storepb.SeriesRequest{ + MinTime: req.Start, + MaxTime: req.End, + SkipChunks: true, + } + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + reqSeriesMatchersNoExtLabels, + nil, + true, + SeriesBatchSize, + s.metrics.seriesFetchDurationSum, + nil, + nil, + nil, + s.enabledLazyExpandedPostings, + s.metrics.lazyExpandedPostingsCount, + s.metrics.lazyExpandedPostingSizeBytes, + s.metrics.lazyExpandedPostingSeriesOverfetchedSizeBytes, + tenant, + ) + defer blockClient.Close() + + if err := blockClient.ExpandPostings( + sortedReqSeriesMatchersNoExtLabels, + seriesLimiter, + ); err != nil { + return err + } + + // Extract given label's value from all series and deduplicate them. + // We don't need to deal with external labels, since they are already added by blockSeries. + values := map[string]struct{}{} + for { + ls, err := blockClient.Recv() + if err == io.EOF { + break + } + if err != nil { + return errors.Wrapf(err, "iterate series for block %s", b.meta.ULID) + } + + if ls.GetWarning() != "" { + return errors.Wrapf(errors.New(ls.GetWarning()), "iterate series for block %s", b.meta.ULID) + } + if ls.GetSeries() == nil { + continue + } + + val := labelpb.ZLabelsToPromLabels(ls.GetSeries().Labels).Get(req.Label) + if val != "" { // Should never be empty since we added labelName!="" matcher to the list of matchers. + values[val] = struct{}{} + } + } + + result = make([]string, 0, len(values)) + for n := range values { + result = append(result, n) + } + sort.Strings(result) + } + + if len(result) > 0 { + mtx.Lock() + sets = append(sets, result) + mtx.Unlock() + } + + return nil + }) + } + + s.mtx.RUnlock() + + if err := g.Wait(); err != nil { + code := codes.Internal + if s, ok := status.FromError(errors.Cause(err)); ok { + code = s.Code() + } + return nil, status.Error(code, err.Error()) + } + + anyHints, err := types.MarshalAny(resHints) + if err != nil { + return nil, status.Error(codes.Unknown, errors.Wrap(err, "marshal label values response hints").Error()) + } + + return &storepb.LabelValuesResponse{ + Values: strutil.MergeSlices(sets...), + Hints: anyHints, + }, nil +} + +// bucketBlockSet holds all blocks of an equal label set. It internally splits +// them up by downsampling resolution and allows querying. +type bucketBlockSet struct { + labels labels.Labels + mtx sync.RWMutex + resolutions []int64 // Available resolution, high to low (in milliseconds). + blocks [][]*bucketBlock // Ordered buckets for the existing resolutions. +} + +// newBucketBlockSet initializes a new set with the known downsampling windows hard-configured. +// The set currently does not support arbitrary ranges. +func newBucketBlockSet(lset labels.Labels) *bucketBlockSet { + return &bucketBlockSet{ + labels: lset, + resolutions: []int64{downsample.ResLevel2, downsample.ResLevel1, downsample.ResLevel0}, + blocks: make([][]*bucketBlock, 3), + } +} + +func (s *bucketBlockSet) add(b *bucketBlock) error { + if !labels.Equal(s.labels, labels.FromMap(b.meta.Thanos.Labels)) { + return errors.New("block's label set does not match set") + } + s.mtx.Lock() + defer s.mtx.Unlock() + + i := int64index(s.resolutions, b.meta.Thanos.Downsample.Resolution) + if i < 0 { + return errors.Errorf("unsupported downsampling resolution %d", b.meta.Thanos.Downsample.Resolution) + } + bs := append(s.blocks[i], b) + s.blocks[i] = bs + + // Always sort blocks by min time, then max time. + sort.Slice(bs, func(j, k int) bool { + if bs[j].meta.MinTime == bs[k].meta.MinTime { + return bs[j].meta.MaxTime < bs[k].meta.MaxTime + } + return bs[j].meta.MinTime < bs[k].meta.MinTime + }) + return nil +} + +func (s *bucketBlockSet) remove(id ulid.ULID) { + s.mtx.Lock() + defer s.mtx.Unlock() + + for i, bs := range s.blocks { + for j, b := range bs { + if b.meta.ULID != id { + continue + } + s.blocks[i] = append(bs[:j], bs[j+1:]...) + return + } + } +} + +func int64index(s []int64, x int64) int { + for i, v := range s { + if v == x { + return i + } + } + return -1 +} + +// getFor returns a time-ordered list of blocks that cover date between mint and maxt. +// Blocks with the biggest resolution possible but not bigger than the given max resolution are returned. +// It supports overlapping blocks. +// +// NOTE: s.blocks are expected to be sorted in minTime order. +func (s *bucketBlockSet) getFor(mint, maxt, maxResolutionMillis int64, blockMatchers []*labels.Matcher) (bs []*bucketBlock) { + if mint > maxt { + return nil + } + + s.mtx.RLock() + defer s.mtx.RUnlock() + + // Find first matching resolution. + i := 0 + for ; i < len(s.resolutions) && s.resolutions[i] > maxResolutionMillis; i++ { + } + + // Fill the given interval with the blocks for the current resolution. + // Our current resolution might not cover all data, so recursively fill the gaps with higher resolution blocks + // if there is any. + start := mint + for _, b := range s.blocks[i] { + if b.meta.MaxTime <= mint { + continue + } + // NOTE: Block intervals are half-open: [b.MinTime, b.MaxTime). + if b.meta.MinTime > maxt { + break + } + + if i+1 < len(s.resolutions) { + bs = append(bs, s.getFor(start, b.meta.MinTime-1, s.resolutions[i+1], blockMatchers)...) + } + + // Include the block in the list of matching ones only if there are no block-level matchers + // or they actually match. + if len(blockMatchers) == 0 || b.matchRelabelLabels(blockMatchers) { + bs = append(bs, b) + } + + start = b.meta.MaxTime + } + + if i+1 < len(s.resolutions) { + bs = append(bs, s.getFor(start, maxt, s.resolutions[i+1], blockMatchers)...) + } + return bs +} + +func (s *bucketBlockSet) getBlock(blockId string) (*bucketBlock, error) { + for _, a := range s.blocks { + for _, b := range a { + if b.meta.ULID.String() == blockId { + return b, nil + } + } + } + return nil, errors.New("block not found") +} + +// labelMatchers verifies whether the block set matches the given matchers and returns a new +// set of matchers that is equivalent when querying data within the block. +func (s *bucketBlockSet) labelMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, bool) { + res := make([]*labels.Matcher, 0, len(matchers)) + + for _, m := range matchers { + v := s.labels.Get(m.Name) + if v == "" { + res = append(res, m) + continue + } + if !m.Matches(v) { + return nil, false + } + } + return res, true +} + +// bucketBlock represents a block that is located in a bucket. It holds intermediate +// state for the block on local disk. +type bucketBlock struct { + logger log.Logger + metrics *bucketStoreMetrics + bkt objstore.BucketReader + meta *metadata.Meta + dir string + indexCache storecache.IndexCache + chunkPool pool.Bytes + extLset labels.Labels + + indexHeaderReader indexheader.Reader + + chunkObjs []string + + pendingReaders sync.WaitGroup + + partitioner Partitioner + + // Block's labels used by block-level matchers to filter blocks to query. These are used to select blocks using + // request hints' BlockMatchers. + relabelLabels labels.Labels + + estimatedMaxChunkSize int + estimatedMaxSeriesSize int +} + +func newBucketBlock( + ctx context.Context, + logger log.Logger, + metrics *bucketStoreMetrics, + meta *metadata.Meta, + bkt objstore.BucketReader, + dir string, + indexCache storecache.IndexCache, + chunkPool pool.Bytes, + indexHeadReader indexheader.Reader, + p Partitioner, + maxSeriesSizeFunc BlockEstimator, + maxChunkSizeFunc BlockEstimator, +) (b *bucketBlock, err error) { + maxSeriesSize := EstimatedMaxSeriesSize + if maxSeriesSizeFunc != nil { + maxSeriesSize = int(maxSeriesSizeFunc(*meta)) + } + maxChunkSize := EstimatedMaxChunkSize + if maxChunkSizeFunc != nil { + maxChunkSize = int(maxChunkSizeFunc(*meta)) + } + b = &bucketBlock{ + logger: logger, + metrics: metrics, + bkt: bkt, + indexCache: indexCache, + chunkPool: chunkPool, + dir: dir, + partitioner: p, + meta: meta, + indexHeaderReader: indexHeadReader, + extLset: labels.FromMap(meta.Thanos.Labels), + // Translate the block's labels and inject the block ID as a label + // to allow to match blocks also by ID. + relabelLabels: append(labels.FromMap(meta.Thanos.Labels), labels.Label{ + Name: block.BlockIDLabel, + Value: meta.ULID.String(), + }), + estimatedMaxSeriesSize: maxSeriesSize, + estimatedMaxChunkSize: maxChunkSize, + } + sort.Sort(b.extLset) + sort.Sort(b.relabelLabels) + + // Get object handles for all chunk files (segment files) from meta.json, if available. + if len(meta.Thanos.SegmentFiles) > 0 { + b.chunkObjs = make([]string, 0, len(meta.Thanos.SegmentFiles)) + + for _, sf := range meta.Thanos.SegmentFiles { + b.chunkObjs = append(b.chunkObjs, path.Join(meta.ULID.String(), block.ChunksDirname, sf)) + } + return b, nil + } + + // Get object handles for all chunk files from storage. + if err = bkt.Iter(ctx, path.Join(meta.ULID.String(), block.ChunksDirname), func(n string) error { + b.chunkObjs = append(b.chunkObjs, n) + return nil + }); err != nil { + return nil, errors.Wrap(err, "list chunk files") + } + return b, nil +} + +func (b *bucketBlock) indexFilename() string { + return path.Join(b.meta.ULID.String(), block.IndexFilename) +} + +func (b *bucketBlock) readIndexRange(ctx context.Context, off, length int64) ([]byte, error) { + r, err := b.bkt.GetRange(ctx, b.indexFilename(), off, length) + if err != nil { + return nil, errors.Wrap(err, "get range reader") + } + defer runutil.CloseWithLogOnErr(b.logger, r, "readIndexRange close range reader") + + // Preallocate the buffer with the exact size so we don't waste allocations + // while progressively growing an initial small buffer. The buffer capacity + // is increased by MinRead to avoid extra allocations due to how ReadFrom() + // internally works. + buf := bytes.NewBuffer(make([]byte, 0, length+bytes.MinRead)) + if _, err := buf.ReadFrom(r); err != nil { + return nil, errors.Wrap(err, "read range") + } + return buf.Bytes(), nil +} + +func (b *bucketBlock) readChunkRange(ctx context.Context, seq int, off, length int64, chunkRanges byteRanges) (*[]byte, error) { + if seq < 0 || seq >= len(b.chunkObjs) { + return nil, errors.Errorf("unknown segment file for index %d", seq) + } + + // Get a reader for the required range. + reader, err := b.bkt.GetRange(ctx, b.chunkObjs[seq], off, length) + if err != nil { + return nil, errors.Wrap(err, "get range reader") + } + defer runutil.CloseWithLogOnErr(b.logger, reader, "readChunkRange close range reader") + + // Get a buffer from the pool. + chunkBuffer, err := b.chunkPool.Get(chunkRanges.size()) + if err != nil { + return nil, errors.Wrap(err, "allocate chunk bytes") + } + + *chunkBuffer, err = readByteRanges(reader, *chunkBuffer, chunkRanges) + if err != nil { + return nil, err + } + + return chunkBuffer, nil +} + +func (b *bucketBlock) chunkRangeReader(ctx context.Context, seq int, off, length int64) (io.ReadCloser, error) { + if seq < 0 || seq >= len(b.chunkObjs) { + return nil, errors.Errorf("unknown segment file for index %d", seq) + } + + return b.bkt.GetRange(ctx, b.chunkObjs[seq], off, length) +} + +func (b *bucketBlock) indexReader() *bucketIndexReader { + b.pendingReaders.Add(1) + return newBucketIndexReader(b) +} + +func (b *bucketBlock) chunkReader() *bucketChunkReader { + b.pendingReaders.Add(1) + return newBucketChunkReader(b) +} + +// matchRelabelLabels verifies whether the block matches the given matchers. +func (b *bucketBlock) matchRelabelLabels(matchers []*labels.Matcher) bool { + for _, m := range matchers { + if !m.Matches(b.relabelLabels.Get(m.Name)) { + return false + } + } + return true +} + +// overlapsClosedInterval returns true if the block overlaps [mint, maxt). +func (b *bucketBlock) overlapsClosedInterval(mint, maxt int64) bool { + // The block itself is a half-open interval + // [b.meta.MinTime, b.meta.MaxTime). + return b.meta.MinTime <= maxt && mint < b.meta.MaxTime +} + +// Close waits for all pending readers to finish and then closes all underlying resources. +func (b *bucketBlock) Close() error { + b.pendingReaders.Wait() + return b.indexHeaderReader.Close() +} + +// bucketIndexReader is a custom index reader (not conforming index.Reader interface) that reads index that is stored in +// object storage without having to fully download it. +type bucketIndexReader struct { + block *bucketBlock + dec *index.Decoder + stats *queryStats + + loadedSeriesMtx sync.Mutex + loadedSeries map[storage.SeriesRef][]byte + + indexVersion int +} + +func newBucketIndexReader(block *bucketBlock) *bucketIndexReader { + r := &bucketIndexReader{ + block: block, + dec: &index.Decoder{ + LookupSymbol: block.indexHeaderReader.LookupSymbol, + }, + stats: &queryStats{}, + loadedSeries: map[storage.SeriesRef][]byte{}, + } + return r +} + +// IndexVersion caches the index header version. +func (r *bucketIndexReader) IndexVersion() (int, error) { + if r.indexVersion != 0 { + return r.indexVersion, nil + } + v, err := r.block.indexHeaderReader.IndexVersion() + if err != nil { + return 0, err + } + r.indexVersion = v + return v, nil +} + +func (r *bucketIndexReader) reset(size int) { + r.loadedSeries = make(map[storage.SeriesRef][]byte, size) +} + +// ExpandedPostings returns postings in expanded list instead of index.Postings. +// This is because we need to have them buffered anyway to perform efficient lookup +// on object storage. +// Found posting IDs (ps) are not strictly required to point to a valid Series, e.g. during +// background garbage collections. +// +// Reminder: A posting is a reference (represented as a uint64) to a series reference, which in turn points to the first +// chunk where the series contains the matching label-value pair for a given block of data. Postings can be fetched by +// single label name=value. +func (r *bucketIndexReader) ExpandedPostings(ctx context.Context, ms sortedMatchers, bytesLimiter BytesLimiter, lazyExpandedPostingEnabled bool, lazyExpandedPostingSizeBytes prometheus.Counter, tenant string) (*lazyExpandedPostings, error) { + // Shortcut the case of `len(postingGroups) == 0`. It will only happen when no + // matchers specified, and we don't need to fetch expanded postings from cache. + if len(ms) == 0 { + return nil, nil + } + + hit, postings, err := r.fetchExpandedPostingsFromCache(ctx, ms, bytesLimiter, tenant) + if err != nil { + return nil, err + } + if hit { + return newLazyExpandedPostings(postings), nil + } + var ( + allRequested = false + hasAdds = false + ) + + postingGroups, err := matchersToPostingGroups(ctx, r.block.indexHeaderReader.LabelValues, ms) + if err != nil { + return nil, errors.Wrap(err, "matchersToPostingGroups") + } + if postingGroups == nil { + r.storeExpandedPostingsToCache(ms, index.EmptyPostings(), 0, tenant) + return nil, nil + } + i := 0 + for _, pg := range postingGroups { + allRequested = allRequested || pg.addAll + hasAdds = hasAdds || len(pg.addKeys) > 0 + + // If a posting group doesn't have any keys, like posting group created + // from `=~".*"`, we don't have to keep the posting group as long as we + // keep track of whether we need to add all postings or not. + if len(pg.addKeys) == 0 && len(pg.removeKeys) == 0 { + continue + } + postingGroups[i] = pg + i++ + } + postingGroups = postingGroups[:i] + + addAllPostings := allRequested && !hasAdds + // We only need special All postings if there are no other adds. If there are, we can skip fetching + // special All postings completely. + if addAllPostings { + // add group with label to fetch "special All postings". + name, value := index.AllPostingsKey() + postingGroups = append(postingGroups, newPostingGroup(true, name, []string{value}, nil)) + } + + ps, err := fetchLazyExpandedPostings(ctx, postingGroups, r, bytesLimiter, addAllPostings, lazyExpandedPostingEnabled, lazyExpandedPostingSizeBytes, tenant) + if err != nil { + return nil, errors.Wrap(err, "fetch and expand postings") + } + // If postings still have matchers to be applied lazily, cache expanded postings after filtering series so skip here. + if !ps.lazyExpanded() { + r.storeExpandedPostingsToCache(ms, index.NewListPostings(ps.postings), len(ps.postings), tenant) + } + + if len(ps.postings) > 0 { + // As of version two all series entries are 16 byte padded. All references + // we get have to account for that to get the correct offset. + version, err := r.IndexVersion() + if err != nil { + return nil, errors.Wrap(err, "get index version") + } + if version >= 2 { + for i, id := range ps.postings { + ps.postings[i] = id * 16 + } + } + } + return ps, nil +} + +// ExpandPostingsWithContext returns the postings expanded as a slice and considers context. +func ExpandPostingsWithContext(ctx context.Context, p index.Postings) (res []storage.SeriesRef, err error) { + for p.Next() { + if ctx.Err() != nil { + return nil, ctx.Err() + } + res = append(res, p.At()) + } + return res, p.Err() +} + +// postingGroup keeps posting keys for one or more matchers with the same label name. Logical result of the group is: +// If addAll is set: special All postings minus postings for removeKeys labels. No need to merge postings for addKeys in this case. +// If addAll is not set: Merge of postings for "addKeys" labels minus postings for removeKeys labels +// This computation happens in ExpandedPostings. +type postingGroup struct { + addAll bool + name string + matchers []*labels.Matcher + addKeys []string + removeKeys []string + cardinality int64 + lazy bool +} + +func newPostingGroup(addAll bool, name string, addKeys, removeKeys []string) *postingGroup { + return &postingGroup{ + name: name, + addAll: addAll, + addKeys: addKeys, + removeKeys: removeKeys, + } +} + +// mergeKeys merges keys from two posting groups and ignores other fields. +func (pg postingGroup) mergeKeys(other *postingGroup) *postingGroup { + if other == nil { + return &pg + } + // This shouldn't happen, but add this as a safeguard. + if pg.name != other.name { + return nil + } + var i, j int + // Both add all, merge remove keys. + if pg.addAll && other.addAll { + // Fast path to not allocate output slice if no remove keys are specified. + // This is possible when matcher is `=~".*"`. + if len(pg.removeKeys) == 0 { + pg.removeKeys = other.removeKeys + return &pg + } else if len(other.removeKeys) == 0 { + return &pg + } + output := make([]string, 0, len(pg.removeKeys)+len(other.removeKeys)) + for i < len(pg.removeKeys) && j < len(other.removeKeys) { + if pg.removeKeys[i] < other.removeKeys[j] { + output = append(output, pg.removeKeys[i]) + i++ + } else if pg.removeKeys[i] > other.removeKeys[j] { + output = append(output, other.removeKeys[j]) + j++ + } else { + output = append(output, pg.removeKeys[i]) + i++ + j++ + } + } + if i < len(pg.removeKeys) { + output = append(output, pg.removeKeys[i:len(pg.removeKeys)]...) + } + if j < len(other.removeKeys) { + output = append(output, other.removeKeys[j:len(other.removeKeys)]...) + } + pg.removeKeys = output + } else if pg.addAll || other.addAll { + // Subtract the remove keys. + toRemove := other + toAdd := &pg + if pg.addAll { + toRemove = &pg + toAdd = other + } + var k int + for i < len(toAdd.addKeys) && j < len(toRemove.removeKeys) { + if toAdd.addKeys[i] < toRemove.removeKeys[j] { + toAdd.addKeys[k] = toAdd.addKeys[i] + k++ + i++ + } else if toAdd.addKeys[i] > toRemove.removeKeys[j] { + j++ + } else { + i++ + j++ + } + } + for i < len(toAdd.addKeys) { + toAdd.addKeys[k] = toAdd.addKeys[i] + i++ + k++ + } + pg.addKeys = toAdd.addKeys[:k] + pg.addAll = false + pg.removeKeys = nil + } else { + addKeys := make([]string, 0, len(pg.addKeys)+len(other.addKeys)) + for i < len(pg.addKeys) && j < len(other.addKeys) { + if pg.addKeys[i] == other.addKeys[j] { + addKeys = append(addKeys, pg.addKeys[i]) + i++ + j++ + } else if pg.addKeys[i] < other.addKeys[j] { + i++ + } else { + j++ + } + } + pg.addKeys = addKeys + } + return &pg +} + +func checkNilPosting(name, value string, p index.Postings) index.Postings { + if p == nil { + // This should not happen. Debug for https://github.com/thanos-io/thanos/issues/874. + return index.ErrPostings(errors.Errorf("postings is nil for {%s=%s}. It was never fetched.", name, value)) + } + return p +} + +func matchersToPostingGroups(ctx context.Context, lvalsFn func(name string) ([]string, error), ms []*labels.Matcher) ([]*postingGroup, error) { + matchersMap := make(map[string]map[string]*labels.Matcher) + for _, m := range ms { + m := m + if _, ok := matchersMap[m.Name]; !ok { + matchersMap[m.Name] = make(map[string]*labels.Matcher) + } + matchersMap[m.Name][m.String()] = m + } + + pgs := make([]*postingGroup, 0, len(matchersMap)) + // NOTE: Derived from tsdb.PostingsForMatchers. + for _, values := range matchersMap { + var ( + mergedPG *postingGroup + pg *postingGroup + vals []string + err error + valuesCached bool + ) + lvalsFunc := lvalsFn + matchers := make([]*labels.Matcher, 0, len(vals)) + // Merge PostingGroups with the same matcher into 1 to + // avoid fetching duplicate postings. + for _, val := range values { + pg, vals, err = toPostingGroup(ctx, lvalsFunc, val) + if err != nil { + return nil, errors.Wrap(err, "toPostingGroup") + } + // Cache label values because label name is the same. + if !valuesCached && vals != nil { + lvals := vals + lvalsFunc = func(_ string) ([]string, error) { + return lvals, nil + } + valuesCached = true + } + + // If this groups adds nothing, it's an empty group. We can shortcut this, since intersection with empty + // postings would return no postings anyway. + // E.g. label="non-existing-value" returns empty group. + if !pg.addAll && len(pg.addKeys) == 0 { + return nil, nil + } + if mergedPG == nil { + mergedPG = pg + } else { + mergedPG = mergedPG.mergeKeys(pg) + } + + // If this groups adds nothing, it's an empty group. We can shortcut this, since intersection with empty + // postings would return no postings anyway. + // E.g. label="non-existing-value" returns empty group. + if !mergedPG.addAll && len(mergedPG.addKeys) == 0 { + return nil, nil + } + matchers = append(matchers, val) + } + // Set and sort matchers to be used when picking up posting fetch strategy. + mergedPG.matchers = newSortedMatchers(matchers) + pgs = append(pgs, mergedPG) + } + slices.SortFunc(pgs, func(a, b *postingGroup) int { + return strings.Compare(a.name, b.name) + }) + return pgs, nil +} + +// NOTE: Derived from tsdb.postingsForMatcher. index.Merge is equivalent to map duplication. +func toPostingGroup(ctx context.Context, lvalsFn func(name string) ([]string, error), m *labels.Matcher) (*postingGroup, []string, error) { + // If the matcher selects an empty value, it selects all the series which don't + // have the label name set too. See: https://github.com/prometheus/prometheus/issues/3575 + // and https://github.com/prometheus/prometheus/pull/3578#issuecomment-351653555. + if m.Matches("") { + var toRemove []string + + // Fast-path for MatchNotRegexp matching. + // Inverse of a MatchNotRegexp is MatchRegexp (double negation). + // Fast-path for set matching. + if m.Type == labels.MatchNotRegexp { + if vals := findSetMatches(m.Value); len(vals) > 0 { + sort.Strings(vals) + return newPostingGroup(true, m.Name, nil, vals), nil, nil + } + } + + // Fast-path for MatchNotEqual matching. + // Inverse of a MatchNotEqual is MatchEqual (double negation). + if m.Type == labels.MatchNotEqual { + return newPostingGroup(true, m.Name, nil, []string{m.Value}), nil, nil + } + + vals, err := lvalsFn(m.Name) + if err != nil { + return nil, nil, err + } + + for _, val := range vals { + if ctx.Err() != nil { + return nil, nil, ctx.Err() + } + if !m.Matches(val) { + toRemove = append(toRemove, val) + } + } + + return newPostingGroup(true, m.Name, nil, toRemove), vals, nil + } + if m.Type == labels.MatchRegexp { + if vals := findSetMatches(m.Value); len(vals) > 0 { + sort.Strings(vals) + return newPostingGroup(false, m.Name, vals, nil), nil, nil + } + } + + // Fast-path for equal matching. + if m.Type == labels.MatchEqual { + return newPostingGroup(false, m.Name, []string{m.Value}, nil), nil, nil + } + + vals, err := lvalsFn(m.Name) + if err != nil { + return nil, nil, err + } + + var toAdd []string + for _, val := range vals { + if ctx.Err() != nil { + return nil, nil, ctx.Err() + } + if m.Matches(val) { + toAdd = append(toAdd, val) + } + } + + return newPostingGroup(false, m.Name, toAdd, nil), vals, nil +} + +type postingPtr struct { + keyID int + ptr index.Range +} + +func (r *bucketIndexReader) fetchExpandedPostingsFromCache(ctx context.Context, ms []*labels.Matcher, bytesLimiter BytesLimiter, tenant string) (bool, []storage.SeriesRef, error) { + dataFromCache, hit := r.block.indexCache.FetchExpandedPostings(ctx, r.block.meta.ULID, ms, tenant) + if !hit { + return false, nil, nil + } + if err := bytesLimiter.Reserve(uint64(len(dataFromCache))); err != nil { + return false, nil, httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while loading expanded postings from index cache: %s", err) + } + r.stats.DataDownloadedSizeSum += units.Base2Bytes(len(dataFromCache)) + r.stats.postingsTouched++ + r.stats.PostingsTouchedSizeSum += units.Base2Bytes(len(dataFromCache)) + p, closeFns, err := r.decodeCachedPostings(dataFromCache) + defer func() { + for _, closeFn := range closeFns { + closeFn() + } + }() + // If failed to decode or expand cached postings, return and expand postings again. + if err != nil { + level.Error(r.block.logger).Log("msg", "failed to decode cached expanded postings, refetch postings", "id", r.block.meta.ULID.String(), "err", err) + return false, nil, nil + } + + ps, err := ExpandPostingsWithContext(ctx, p) + if err != nil { + level.Error(r.block.logger).Log("msg", "failed to expand cached expanded postings, refetch postings", "id", r.block.meta.ULID.String(), "err", err) + return false, nil, nil + } + + if len(ps) > 0 { + // As of version two all series entries are 16 byte padded. All references + // we get have to account for that to get the correct offset. + version, err := r.block.indexHeaderReader.IndexVersion() + if err != nil { + return false, nil, errors.Wrap(err, "get index version") + } + if version >= 2 { + for i, id := range ps { + ps[i] = id * 16 + } + } + } + return true, ps, nil +} + +func (r *bucketIndexReader) storeExpandedPostingsToCache(ms []*labels.Matcher, ps index.Postings, length int, tenant string) { + // Encode postings to cache. We compress and cache postings before adding + // 16 bytes padding in order to make compressed size smaller. + dataToCache, compressionDuration, compressionErrors, compressedSize := r.encodePostingsToCache(ps, length) + r.stats.cachedPostingsCompressions++ + r.stats.cachedPostingsCompressionErrors += compressionErrors + r.stats.CachedPostingsCompressionTimeSum += compressionDuration + r.stats.CachedPostingsCompressedSizeSum += units.Base2Bytes(compressedSize) + r.stats.CachedPostingsOriginalSizeSum += units.Base2Bytes(length * 4) // Estimate the posting list size. + r.block.indexCache.StoreExpandedPostings(r.block.meta.ULID, ms, dataToCache, tenant) +} + +var bufioReaderPool = sync.Pool{ + New: func() any { + return bufio.NewReader(nil) + }, +} + +// fetchPostings fill postings requested by posting groups. +// It returns one posting for each key, in the same order. +// If postings for given key is not fetched, entry at given index will be nil. +func (r *bucketIndexReader) fetchPostings(ctx context.Context, keys []labels.Label, bytesLimiter BytesLimiter, tenant string) ([]index.Postings, []func(), error) { + var closeFns []func() + + timer := prometheus.NewTimer(r.block.metrics.postingsFetchDuration.WithLabelValues(tenant)) + defer timer.ObserveDuration() + + var ptrs []postingPtr + + output := make([]index.Postings, len(keys)) + + // Fetch postings from the cache with a single call. + fromCache, _ := r.block.indexCache.FetchMultiPostings(ctx, r.block.meta.ULID, keys, tenant) + for _, dataFromCache := range fromCache { + if err := bytesLimiter.Reserve(uint64(len(dataFromCache))); err != nil { + return nil, closeFns, httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while loading postings from index cache: %s", err) + } + r.stats.DataDownloadedSizeSum += units.Base2Bytes(len(dataFromCache)) + } + + // Iterate over all groups and fetch posting from cache. + // If we have a miss, mark key to be fetched in `ptrs` slice. + // Overlaps are well handled by partitioner, so we don't need to deduplicate keys. + for ix, key := range keys { + if err := ctx.Err(); err != nil { + return nil, closeFns, err + } + // Get postings for the given key from cache first. + if b, ok := fromCache[key]; ok { + r.stats.postingsTouched++ + r.stats.PostingsTouchedSizeSum += units.Base2Bytes(len(b)) + + l, closer, err := r.decodeCachedPostings(b) + if err != nil { + return nil, closeFns, errors.Wrap(err, "decode postings") + } + output[ix] = l + closeFns = append(closeFns, closer...) + continue + } + + // Cache miss; save pointer for actual posting in index stored in object store. + ptr, err := r.block.indexHeaderReader.PostingsOffset(key.Name, key.Value) + if err == indexheader.NotFoundRangeErr { + // This block does not have any posting for given key. + output[ix] = index.EmptyPostings() + continue + } + + if err != nil { + return nil, closeFns, errors.Wrap(err, "index header PostingsOffset") + } + + r.stats.postingsToFetch++ + ptrs = append(ptrs, postingPtr{ptr: ptr, keyID: ix}) + } + + sort.Slice(ptrs, func(i, j int) bool { + return ptrs[i].ptr.Start < ptrs[j].ptr.Start + }) + + // TODO(bwplotka): Asses how large in worst case scenario this can be. (e.g fetch for AllPostingsKeys) + // Consider sub split if too big. + parts := r.block.partitioner.Partition(len(ptrs), func(i int) (start, end uint64) { + return uint64(ptrs[i].ptr.Start), uint64(ptrs[i].ptr.End) + }) + + for _, part := range parts { + start := int64(part.Start) + length := int64(part.End) - start + + if err := bytesLimiter.Reserve(uint64(length)); err != nil { + return nil, closeFns, httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while fetching postings: %s", err) + } + r.stats.DataDownloadedSizeSum += units.Base2Bytes(length) + } + + g, ctx := errgroup.WithContext(ctx) + for _, part := range parts { + i, j := part.ElemRng[0], part.ElemRng[1] + + start := int64(part.Start) + // We assume index does not have any ptrs that has 0 length. + length := int64(part.End) - start + + // Fetch from object storage concurrently and update stats and posting list. + g.Go(func() error { + begin := time.Now() + stats := new(queryStats) + defer func() { + r.stats.merge(stats) + }() + + brdr := bufioReaderPool.Get().(*bufio.Reader) + defer bufioReaderPool.Put(brdr) + + partReader, err := r.block.bkt.GetRange(ctx, r.block.indexFilename(), start, length) + if err != nil { + return errors.Wrap(err, "read postings range") + } + defer runutil.CloseWithLogOnErr(r.block.logger, partReader, "readIndexRange close range reader") + brdr.Reset(partReader) + + rdr := newPostingsReaderBuilder(ctx, brdr, ptrs[i:j], start, length) + + stats.postingsFetchCount++ + stats.postingsFetched += j - i + stats.PostingsFetchedSizeSum += units.Base2Bytes(int(length)) + + for rdr.Next() { + diffVarintPostings, postingsCount, keyID := rdr.AtDiffVarint() + + output[keyID] = newDiffVarintPostings(diffVarintPostings, nil) + + startCompression := time.Now() + dataToCache, err := snappyStreamedEncode(int(postingsCount), diffVarintPostings) + if err != nil { + stats.cachedPostingsCompressionErrors += 1 + return errors.Wrap(err, "encoding with snappy") + } + + stats.postingsTouched++ + stats.PostingsTouchedSizeSum += units.Base2Bytes(int(len(diffVarintPostings))) + stats.cachedPostingsCompressions += 1 + stats.CachedPostingsOriginalSizeSum += units.Base2Bytes(len(diffVarintPostings)) + stats.CachedPostingsCompressedSizeSum += units.Base2Bytes(len(dataToCache)) + stats.CachedPostingsCompressionTimeSum += time.Since(startCompression) + + r.block.indexCache.StorePostings(r.block.meta.ULID, keys[keyID], dataToCache, tenant) + } + + stats.PostingsFetchDurationSum += time.Since(begin) + + if err := rdr.Error(); err != nil { + return errors.Wrap(err, "reading postings") + } + return nil + }) + } + + return output, closeFns, g.Wait() +} + +func (r *bucketIndexReader) decodeCachedPostings(b []byte) (index.Postings, []func(), error) { + // Even if this instance is not using compression, there may be compressed + // entries in the cache written by other stores. + var ( + l index.Postings + err error + closeFns []func() + ) + if isDiffVarintSnappyEncodedPostings(b) || isDiffVarintSnappyStreamedEncodedPostings(b) { + s := time.Now() + l, err = decodePostings(b) + r.stats.cachedPostingsDecompressions += 1 + r.stats.CachedPostingsDecompressionTimeSum += time.Since(s) + if err != nil { + r.stats.cachedPostingsDecompressionErrors += 1 + } else { + closeFns = append(closeFns, l.(closeablePostings).close) + } + } else { + _, l, err = r.dec.Postings(b) + } + return l, closeFns, err +} + +func (r *bucketIndexReader) encodePostingsToCache(p index.Postings, length int) ([]byte, time.Duration, int, int) { + var dataToCache []byte + compressionTime := time.Duration(0) + compressionErrors, compressedSize := 0, 0 + s := time.Now() + data, err := diffVarintSnappyStreamedEncode(p, length) + compressionTime = time.Since(s) + if err == nil { + dataToCache = data + compressedSize = len(data) + } else { + compressionErrors = 1 + } + return dataToCache, compressionTime, compressionErrors, compressedSize +} + +// bigEndianPostings implements the Postings interface over a byte stream of +// big endian numbers. +type bigEndianPostings struct { + list []byte + cur uint32 +} + +// TODO(bwplotka): Expose those inside Prometheus. +func newBigEndianPostings(list []byte) *bigEndianPostings { + return &bigEndianPostings{list: list} +} + +func (it *bigEndianPostings) At() storage.SeriesRef { + return storage.SeriesRef(it.cur) +} + +func (it *bigEndianPostings) Next() bool { + if len(it.list) >= 4 { + it.cur = binary.BigEndian.Uint32(it.list) + it.list = it.list[4:] + return true + } + return false +} + +func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool { + if storage.SeriesRef(it.cur) >= x { + return true + } + + num := len(it.list) / 4 + // Do binary search between current position and end. + i := sort.Search(num, func(i int) bool { + return binary.BigEndian.Uint32(it.list[i*4:]) >= uint32(x) + }) + if i < num { + j := i * 4 + it.cur = binary.BigEndian.Uint32(it.list[j:]) + it.list = it.list[j+4:] + return true + } + it.list = nil + return false +} + +func (it *bigEndianPostings) Err() error { + return nil +} + +// Returns number of remaining postings values. +func (it *bigEndianPostings) length() int { + return len(it.list) / 4 +} + +func (r *bucketIndexReader) PreloadSeries(ctx context.Context, ids []storage.SeriesRef, bytesLimiter BytesLimiter, tenant string) error { + timer := prometheus.NewTimer(r.block.metrics.seriesFetchDuration.WithLabelValues(tenant)) + defer func() { + d := timer.ObserveDuration() + r.stats.SeriesDownloadLatencySum += d + }() + + // Load series from cache, overwriting the list of ids to preload + // with the missing ones. + fromCache, ids := r.block.indexCache.FetchMultiSeries(ctx, r.block.meta.ULID, ids, tenant) + for id, b := range fromCache { + r.loadedSeries[id] = b + if err := bytesLimiter.Reserve(uint64(len(b))); err != nil { + return httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while loading series from index cache: %s", err) + } + r.stats.DataDownloadedSizeSum += units.Base2Bytes(len(b)) + } + + parts := r.block.partitioner.Partition(len(ids), func(i int) (start, end uint64) { + return uint64(ids[i]), uint64(ids[i]) + uint64(r.block.estimatedMaxSeriesSize) + }) + + g, ctx := errgroup.WithContext(ctx) + for _, p := range parts { + s, e := p.Start, p.End + i, j := p.ElemRng[0], p.ElemRng[1] + + g.Go(func() error { + return r.loadSeries(ctx, ids[i:j], false, s, e, bytesLimiter, tenant) + }) + } + return g.Wait() +} + +func (r *bucketIndexReader) loadSeries(ctx context.Context, ids []storage.SeriesRef, refetch bool, start, end uint64, bytesLimiter BytesLimiter, tenant string) error { + begin := time.Now() + stats := new(queryStats) + defer func() { + r.stats.merge(stats) + }() + + if bytesLimiter != nil { + if err := bytesLimiter.Reserve(uint64(end - start)); err != nil { + return httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while fetching series: %s", err) + } + stats.DataDownloadedSizeSum += units.Base2Bytes(end - start) + } + + b, err := r.block.readIndexRange(ctx, int64(start), int64(end-start)) + if err != nil { + return errors.Wrap(err, "read series range") + } + + stats.seriesFetchCount++ + stats.seriesFetched += len(ids) + stats.SeriesFetchDurationSum += time.Since(begin) + stats.SeriesFetchedSizeSum += units.Base2Bytes(int(end - start)) + + for i, id := range ids { + c := b[uint64(id)-start:] + + l, n := binary.Uvarint(c) + if n < 1 { + return errors.New("reading series length failed") + } + if len(c) < n+int(l) { + if i == 0 && refetch { + return errors.Errorf("invalid remaining size, even after refetch, remaining: %d, expected %d", len(c), n+int(l)) + } + + // Inefficient, but should be rare. + r.block.metrics.seriesRefetches.WithLabelValues(tenant).Inc() + level.Warn(r.block.logger).Log("msg", "series size exceeded expected size; refetching", "id", id, "series length", n+int(l), "maxSeriesSize", r.block.estimatedMaxSeriesSize) + + // Fetch plus to get the size of next one if exists. + return r.loadSeries(ctx, ids[i:], true, uint64(id), uint64(id)+uint64(n+int(l)+1), bytesLimiter, tenant) + } + c = c[n : n+int(l)] + r.loadedSeriesMtx.Lock() + r.loadedSeries[id] = c + r.loadedSeriesMtx.Unlock() + + r.block.indexCache.StoreSeries(r.block.meta.ULID, id, c, tenant) + } + return nil +} + +type symbolizedLabel struct { + name, value uint32 +} + +// LoadSeriesForTime populates the given symbolized labels for the series identified by the reference if at least one chunk is within +// time selection. +// LoadSeriesForTime also populates chunk metas slices if skipChunks if set to false. Chunks are also limited by the given time selection. +// LoadSeriesForTime returns false, when there are no series data for given time range. +// +// Error is returned on decoding error or if the reference does not resolve to a known series. +func (r *bucketIndexReader) LoadSeriesForTime(ref storage.SeriesRef, lset *[]symbolizedLabel, chks *[]chunks.Meta, skipChunks bool, mint, maxt int64) (ok bool, err error) { + b, ok := r.loadedSeries[ref] + if !ok { + return false, errors.Errorf("series %d not found", ref) + } + + r.stats.seriesTouched++ + r.stats.SeriesTouchedSizeSum += units.Base2Bytes(len(b)) + return decodeSeriesForTime(b, lset, chks, skipChunks, mint, maxt) +} + +// Close released the underlying resources of the reader. +func (r *bucketIndexReader) Close() error { + r.block.pendingReaders.Done() + return nil +} + +// LookupLabelsSymbols allows populates label set strings from symbolized label set. +func (r *bucketIndexReader) LookupLabelsSymbols(ctx context.Context, symbolized []symbolizedLabel, lbls *labels.Labels) error { + *lbls = (*lbls)[:0] + for _, s := range symbolized { + ln, err := r.dec.LookupSymbol(ctx, s.name) + if err != nil { + return errors.Wrap(err, "lookup label name") + } + lv, err := r.dec.LookupSymbol(ctx, s.value) + if err != nil { + return errors.Wrap(err, "lookup label value") + } + *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) + } + return nil +} + +// decodeSeriesForTime decodes a series entry from the given byte slice decoding only chunk metas that are within given min and max time. +// If skipChunks is specified decodeSeriesForTime does not return any chunks, but only labels and only if at least single chunk is within time range. +// decodeSeriesForTime returns false, when there are no series data for given time range. +func decodeSeriesForTime(b []byte, lset *[]symbolizedLabel, chks *[]chunks.Meta, skipChunks bool, selectMint, selectMaxt int64) (ok bool, err error) { + *lset = (*lset)[:0] + *chks = (*chks)[:0] + + d := encoding.Decbuf{B: b} + + // Read labels without looking up symbols. + k := d.Uvarint() + for i := 0; i < k; i++ { + lno := uint32(d.Uvarint()) + lvo := uint32(d.Uvarint()) + *lset = append(*lset, symbolizedLabel{name: lno, value: lvo}) + } + // Read the chunks meta data. + k = d.Uvarint() + if k == 0 { + return false, d.Err() + } + + // First t0 is absolute, rest is just diff so different type is used (Uvarint64). + mint := d.Varint64() + maxt := int64(d.Uvarint64()) + mint + // Similar for first ref. + ref := int64(d.Uvarint64()) + + for i := 0; i < k; i++ { + if i > 0 { + mint += int64(d.Uvarint64()) + maxt = int64(d.Uvarint64()) + mint + ref += d.Varint64() + } + + if mint > selectMaxt { + break + } + + if maxt >= selectMint { + // Found a chunk. + if skipChunks { + // We are not interested in chunks and we know there is at least one, that's enough to return series. + return true, nil + } + + *chks = append(*chks, chunks.Meta{ + Ref: chunks.ChunkRef(ref), + MinTime: mint, + MaxTime: maxt, + }) + } + + mint = maxt + } + return len(*chks) > 0, d.Err() +} + +type queryStats struct { + mtx sync.Mutex + + blocksQueried int + postingsTouched int + PostingsTouchedSizeSum units.Base2Bytes + postingsToFetch int + postingsFetched int + PostingsFetchedSizeSum units.Base2Bytes + postingsFetchCount int + PostingsFetchDurationSum time.Duration + + cachedPostingsCompressions int + cachedPostingsCompressionErrors int + CachedPostingsOriginalSizeSum units.Base2Bytes + CachedPostingsCompressedSizeSum units.Base2Bytes + CachedPostingsCompressionTimeSum time.Duration + cachedPostingsDecompressions int + cachedPostingsDecompressionErrors int + CachedPostingsDecompressionTimeSum time.Duration + + seriesTouched int + SeriesTouchedSizeSum units.Base2Bytes + seriesFetched int + SeriesFetchedSizeSum units.Base2Bytes + seriesFetchCount int + SeriesFetchDurationSum time.Duration + SeriesDownloadLatencySum time.Duration + + chunksTouched int + ChunksTouchedSizeSum units.Base2Bytes + chunksFetched int + ChunksFetchedSizeSum units.Base2Bytes + chunksFetchCount int + ChunksFetchDurationSum time.Duration + ChunksDownloadLatencySum time.Duration + + GetAllDuration time.Duration + mergedSeriesCount int + mergedChunksCount int + MergeDuration time.Duration + + DataDownloadedSizeSum units.Base2Bytes +} + +func (s *queryStats) merge(o *queryStats) { + s.mtx.Lock() + defer s.mtx.Unlock() + + s.blocksQueried += o.blocksQueried + + s.postingsToFetch += o.postingsToFetch + s.postingsTouched += o.postingsTouched + s.PostingsTouchedSizeSum += o.PostingsTouchedSizeSum + s.postingsFetched += o.postingsFetched + s.PostingsFetchedSizeSum += o.PostingsFetchedSizeSum + s.postingsFetchCount += o.postingsFetchCount + s.PostingsFetchDurationSum += o.PostingsFetchDurationSum + + s.cachedPostingsCompressions += o.cachedPostingsCompressions + s.cachedPostingsCompressionErrors += o.cachedPostingsCompressionErrors + s.CachedPostingsOriginalSizeSum += o.CachedPostingsOriginalSizeSum + s.CachedPostingsCompressedSizeSum += o.CachedPostingsCompressedSizeSum + s.CachedPostingsCompressionTimeSum += o.CachedPostingsCompressionTimeSum + s.cachedPostingsDecompressions += o.cachedPostingsDecompressions + s.cachedPostingsDecompressionErrors += o.cachedPostingsDecompressionErrors + s.CachedPostingsDecompressionTimeSum += o.CachedPostingsDecompressionTimeSum + + s.seriesTouched += o.seriesTouched + s.SeriesTouchedSizeSum += o.SeriesTouchedSizeSum + s.seriesFetched += o.seriesFetched + s.SeriesFetchedSizeSum += o.SeriesFetchedSizeSum + s.seriesFetchCount += o.seriesFetchCount + s.SeriesFetchDurationSum += o.SeriesFetchDurationSum + s.SeriesDownloadLatencySum += o.SeriesDownloadLatencySum + + s.chunksTouched += o.chunksTouched + s.ChunksTouchedSizeSum += o.ChunksTouchedSizeSum + s.chunksFetched += o.chunksFetched + s.ChunksFetchedSizeSum += o.ChunksFetchedSizeSum + s.chunksFetchCount += o.chunksFetchCount + s.ChunksFetchDurationSum += o.ChunksFetchDurationSum + s.ChunksDownloadLatencySum += o.ChunksDownloadLatencySum + + s.GetAllDuration += o.GetAllDuration + s.mergedSeriesCount += o.mergedSeriesCount + s.mergedChunksCount += o.mergedChunksCount + s.MergeDuration += o.MergeDuration + + s.DataDownloadedSizeSum += o.DataDownloadedSizeSum +} + +func (s *queryStats) toHints() *hintspb.QueryStats { + return &hintspb.QueryStats{ + BlocksQueried: int64(s.blocksQueried), + PostingsTouched: int64(s.postingsTouched), + PostingsTouchedSizeSum: int64(s.PostingsTouchedSizeSum), + PostingsToFetch: int64(s.postingsToFetch), + PostingsFetched: int64(s.postingsFetched), + PostingsFetchedSizeSum: int64(s.PostingsFetchedSizeSum), + PostingsFetchCount: int64(s.postingsFetchCount), + SeriesTouched: int64(s.seriesTouched), + SeriesTouchedSizeSum: int64(s.SeriesTouchedSizeSum), + SeriesFetched: int64(s.seriesFetched), + SeriesFetchedSizeSum: int64(s.SeriesFetchedSizeSum), + SeriesFetchCount: int64(s.seriesFetchCount), + ChunksTouched: int64(s.chunksTouched), + ChunksTouchedSizeSum: int64(s.ChunksTouchedSizeSum), + ChunksFetched: int64(s.chunksFetched), + ChunksFetchedSizeSum: int64(s.ChunksFetchedSizeSum), + ChunksFetchCount: int64(s.chunksFetchCount), + MergedSeriesCount: int64(s.mergedSeriesCount), + MergedChunksCount: int64(s.mergedChunksCount), + DataDownloadedSizeSum: int64(s.DataDownloadedSizeSum), + GetAllDuration: s.GetAllDuration, + MergeDuration: s.MergeDuration, + } +} + +// NewDefaultChunkBytesPool returns a chunk bytes pool with default settings. +func NewDefaultChunkBytesPool(maxChunkPoolBytes uint64) (pool.Bytes, error) { + return pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, maxChunkPoolBytes) +} diff --git a/pkg/storegateway/bucket_store_inmemory_server.go b/pkg/storegateway/bucket_store_inmemory_server.go index d3902d5b23..ddb37504cd 100644 --- a/pkg/storegateway/bucket_store_inmemory_server.go +++ b/pkg/storegateway/bucket_store_inmemory_server.go @@ -3,11 +3,12 @@ package storegateway import ( "context" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/gogo/protobuf/types" "github.com/pkg/errors" "github.com/prometheus/prometheus/util/annotations" "github.com/thanos-io/thanos/pkg/store/hintspb" - "github.com/thanos-io/thanos/pkg/store/storepb" ) // bucketStoreSeriesServer is a fake in-memory gRPC server used to @@ -19,7 +20,7 @@ type bucketStoreSeriesServer struct { ctx context.Context - SeriesSet []*storepb.Series + SeriesSet []*typespb.Series Warnings annotations.Annotations Hints hintspb.SeriesResponseHints } @@ -50,7 +51,7 @@ func (s *bucketStoreSeriesServer) Send(r *storepb.SeriesResponse) error { return errors.Wrap(err, "marshal received series") } - copiedSeries := &storepb.Series{} + copiedSeries := &typespb.Series{} if err = copiedSeries.Unmarshal(recvSeriesData); err != nil { return errors.Wrap(err, "unmarshal received series") } diff --git a/pkg/storegateway/bucket_stores.go b/pkg/storegateway/bucket_stores.go index 228099e1af..1601d89921 100644 --- a/pkg/storegateway/bucket_stores.go +++ b/pkg/storegateway/bucket_stores.go @@ -11,6 +11,7 @@ import ( "sync" "time" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/oklog/ulid" @@ -24,9 +25,7 @@ import ( "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/gate" "github.com/thanos-io/thanos/pkg/pool" - "github.com/thanos-io/thanos/pkg/store" storecache "github.com/thanos-io/thanos/pkg/store/cache" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/logging" "google.golang.org/grpc/codes" @@ -55,19 +54,20 @@ type BucketStores struct { // Index cache shared across all tenants. indexCache storecache.IndexCache + loadIndex bool // Chunks bytes pool shared across all tenants. chunksPool pool.Bytes // Partitioner shared across all tenants. - partitioner store.Partitioner + partitioner Partitioner // Gate used to limit query concurrency across all tenants. queryGate gate.Gate // Keeps a bucket store for each tenant. storesMu sync.RWMutex - stores map[string]*store.BucketStore + stores map[string]*BucketStore // Keeps the last sync error for the bucket store for each tenant. storesErrorsMu sync.RWMutex @@ -87,7 +87,7 @@ type BucketStores struct { var ErrTooManyInflightRequests = status.Error(codes.ResourceExhausted, "too many inflight requests in store gateway") // NewBucketStores makes a new BucketStores. -func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStrategy, bucketClient objstore.Bucket, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*BucketStores, error) { +func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStrategy, bucketClient objstore.Bucket, loadIndex bool, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*BucketStores, error) { cachingBucket, err := tsdb.CreateCachingBucket(cfg.BucketStore.ChunksCache, cfg.BucketStore.MetadataCache, bucketClient, logger, reg) if err != nil { return nil, errors.Wrapf(err, "create caching bucket") @@ -104,10 +104,11 @@ func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStra u := &BucketStores{ logger: logger, cfg: cfg, + loadIndex: loadIndex, limits: limits, bucket: cachingBucket, shardingStrategy: shardingStrategy, - stores: map[string]*store.BucketStore{}, + stores: map[string]*BucketStore{}, storesErrors: map[string]error{}, logLevel: logLevel, bucketStoreMetrics: NewBucketStoreMetrics(), @@ -154,7 +155,7 @@ func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStra func (u *BucketStores) InitialSync(ctx context.Context) error { level.Info(u.logger).Log("msg", "synchronizing TSDB blocks for all users") - if err := u.syncUsersBlocksWithRetries(ctx, func(ctx context.Context, s *store.BucketStore) error { + if err := u.syncUsersBlocksWithRetries(ctx, func(ctx context.Context, s *BucketStore) error { return s.InitialSync(ctx) }); err != nil { level.Warn(u.logger).Log("msg", "failed to synchronize TSDB blocks", "err", err) @@ -167,12 +168,12 @@ func (u *BucketStores) InitialSync(ctx context.Context) error { // SyncBlocks synchronizes the stores state with the Bucket store for every user. func (u *BucketStores) SyncBlocks(ctx context.Context) error { - return u.syncUsersBlocksWithRetries(ctx, func(ctx context.Context, s *store.BucketStore) error { + return u.syncUsersBlocksWithRetries(ctx, func(ctx context.Context, s *BucketStore) error { return s.SyncBlocks(ctx) }) } -func (u *BucketStores) syncUsersBlocksWithRetries(ctx context.Context, f func(context.Context, *store.BucketStore) error) error { +func (u *BucketStores) syncUsersBlocksWithRetries(ctx context.Context, f func(context.Context, *BucketStore) error) error { retries := backoff.New(ctx, backoff.Config{ MinBackoff: 1 * time.Second, MaxBackoff: 10 * time.Second, @@ -196,7 +197,7 @@ func (u *BucketStores) syncUsersBlocksWithRetries(ctx context.Context, f func(co return lastErr } -func (u *BucketStores) syncUsersBlocks(ctx context.Context, f func(context.Context, *store.BucketStore) error) (returnErr error) { +func (u *BucketStores) syncUsersBlocks(ctx context.Context, f func(context.Context, *BucketStore) error) (returnErr error) { defer func(start time.Time) { u.syncTimes.Observe(time.Since(start).Seconds()) if returnErr == nil { @@ -206,7 +207,7 @@ func (u *BucketStores) syncUsersBlocks(ctx context.Context, f func(context.Conte type job struct { userID string - store *store.BucketStore + store *BucketStore } wg := &sync.WaitGroup{} @@ -338,6 +339,86 @@ func (u *BucketStores) Series(req *storepb.SeriesRequest, srv storepb.Store_Seri return err } +// Series makes a series request to the underlying user bucket store. +func (u *BucketStores) Select(req *storepb.SelectRequest, srv storepb.Store_SelectServer) error { + spanLog, spanCtx := spanlogger.New(srv.Context(), "BucketStores.Series") + defer spanLog.Span.Finish() + + userID := getUserIDFromGRPCContext(spanCtx) + if userID == "" { + return fmt.Errorf("no userID") + } + + err := u.getStoreError(userID) + userBkt := bucket.NewUserBucketClient(userID, u.bucket, u.limits) + if err != nil { + if cortex_errors.ErrorIs(err, userBkt.IsAccessDeniedErr) { + return httpgrpc.Errorf(int(codes.PermissionDenied), "store error: %s", err) + } + + return err + } + + store := u.getStore(userID) + if store == nil { + return nil + } + + maxInflightRequests := u.cfg.BucketStore.MaxInflightRequests + if maxInflightRequests > 0 { + if u.getInflightRequestCnt() >= maxInflightRequests { + return ErrTooManyInflightRequests + } + + u.incrementInflightRequestCnt() + defer u.decrementInflightRequestCnt() + } + + err = store.Select(req, srv) + + return err +} + +// Chunks returns chunks +func (u *BucketStores) Chunks(srv storepb.ChunkStore_ChunksServer) error { + spanLog, spanCtx := spanlogger.New(srv.Context(), "BucketStores.Series") + defer spanLog.Span.Finish() + + userID := getUserIDFromGRPCContext(spanCtx) + if userID == "" { + return fmt.Errorf("no userID") + } + + err := u.getStoreError(userID) + userBkt := bucket.NewUserBucketClient(userID, u.bucket, u.limits) + if err != nil { + if cortex_errors.ErrorIs(err, userBkt.IsAccessDeniedErr) { + return httpgrpc.Errorf(int(codes.PermissionDenied), "store error: %s", err) + } + + return err + } + + store := u.getStore(userID) + if store == nil { + return nil + } + + maxInflightRequests := u.cfg.BucketStore.MaxInflightRequests + if maxInflightRequests > 0 { + if u.getInflightRequestCnt() >= maxInflightRequests { + return ErrTooManyInflightRequests + } + + u.incrementInflightRequestCnt() + defer u.decrementInflightRequestCnt() + } + + err = store.Chunks(srv) + + return err +} + func (u *BucketStores) getInflightRequestCnt() int { u.inflightRequestMu.RLock() defer u.inflightRequestMu.RUnlock() @@ -430,7 +511,7 @@ func (u *BucketStores) scanUsers(ctx context.Context) ([]string, error) { return users, err } -func (u *BucketStores) getStore(userID string) *store.BucketStore { +func (u *BucketStores) getStore(userID string) *BucketStore { u.storesMu.RLock() defer u.storesMu.RUnlock() return u.stores[userID] @@ -479,7 +560,7 @@ func (u *BucketStores) closeEmptyBucketStore(userID string) error { return bs.Close() } -func isEmptyBucketStore(bs *store.BucketStore) bool { +func isEmptyBucketStore(bs *BucketStore) bool { min, max := bs.TimeRange() return min == math.MaxInt64 && max == math.MinInt64 } @@ -488,7 +569,7 @@ func (u *BucketStores) syncDirForUser(userID string) string { return filepath.Join(u.cfg.BucketStore.SyncDir, userID) } -func (u *BucketStores) getOrCreateStore(userID string) (*store.BucketStore, error) { +func (u *BucketStores) getOrCreateStore(userID string) (*BucketStore, error) { // Check if the store already exists. bs := u.getStore(userID) if bs != nil { @@ -566,35 +647,35 @@ func (u *BucketStores) getOrCreateStore(userID string) (*store.BucketStore, erro } bucketStoreReg := prometheus.NewRegistry() - bucketStoreOpts := []store.BucketStoreOption{ - store.WithLogger(userLogger), - store.WithRegistry(bucketStoreReg), - store.WithIndexCache(u.indexCache), - store.WithQueryGate(u.queryGate), - store.WithChunkPool(u.chunksPool), - store.WithSeriesBatchSize(u.cfg.BucketStore.SeriesBatchSize), - store.WithBlockEstimatedMaxChunkFunc(func(m thanos_metadata.Meta) uint64 { + bucketStoreOpts := []BucketStoreOption{ + WithLogger(userLogger), + WithRegistry(bucketStoreReg), + WithIndexCache(u.indexCache), + WithQueryGate(u.queryGate), + WithChunkPool(u.chunksPool), + WithSeriesBatchSize(u.cfg.BucketStore.SeriesBatchSize), + WithBlockEstimatedMaxChunkFunc(func(m thanos_metadata.Meta) uint64 { if m.Thanos.IndexStats.ChunkMaxSize > 0 && uint64(m.Thanos.IndexStats.ChunkMaxSize) < u.cfg.BucketStore.EstimatedMaxChunkSizeBytes { return uint64(m.Thanos.IndexStats.ChunkMaxSize) } return u.cfg.BucketStore.EstimatedMaxChunkSizeBytes }), - store.WithBlockEstimatedMaxSeriesFunc(func(m thanos_metadata.Meta) uint64 { + WithBlockEstimatedMaxSeriesFunc(func(m thanos_metadata.Meta) uint64 { if m.Thanos.IndexStats.SeriesMaxSize > 0 && uint64(m.Thanos.IndexStats.SeriesMaxSize) < u.cfg.BucketStore.EstimatedMaxSeriesSizeBytes { return uint64(m.Thanos.IndexStats.SeriesMaxSize) } return u.cfg.BucketStore.EstimatedMaxSeriesSizeBytes }), - store.WithLazyExpandedPostings(u.cfg.BucketStore.LazyExpandedPostingsEnabled), - store.WithDontResort(true), // Cortex doesn't need to resort series in store gateway. + WithLazyExpandedPostings(u.cfg.BucketStore.LazyExpandedPostingsEnabled), + WithDontResort(true), // Cortex doesn't need to resort series in store gateway. } if u.logLevel.String() == "debug" { - bucketStoreOpts = append(bucketStoreOpts, store.WithDebugLogging()) + bucketStoreOpts = append(bucketStoreOpts, WithDebugLogging()) } - bs, err := store.NewBucketStore( + bs, err := NewBucketStore( userBkt, fetcher, u.syncDirForUser(userID), @@ -606,6 +687,7 @@ func (u *BucketStores) getOrCreateStore(userID string) (*store.BucketStore, erro false, // No need to enable backward compatibility with Thanos pre 0.8.0 queriers u.cfg.BucketStore.PostingOffsetsInMemSampling, true, // Enable series hints. + u.loadIndex, u.cfg.BucketStore.IndexHeaderLazyLoadingEnabled, u.cfg.BucketStore.IndexHeaderLazyLoadingIdleTimeout, bucketStoreOpts..., @@ -714,7 +796,7 @@ func (s spanSeriesServer) Context() context.Context { } type limiter struct { - limiter *store.Limiter + limiter *Limiter } func (c *limiter) Reserve(num uint64) error { @@ -726,32 +808,32 @@ func (c *limiter) Reserve(num uint64) error { return nil } -func newChunksLimiterFactory(limits *validation.Overrides, userID string) store.ChunksLimiterFactory { - return func(failedCounter prometheus.Counter) store.ChunksLimiter { +func newChunksLimiterFactory(limits *validation.Overrides, userID string) ChunksLimiterFactory { + return func(failedCounter prometheus.Counter) ChunksLimiter { // Since limit overrides could be live reloaded, we have to get the current user's limit // each time a new limiter is instantiated. return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxChunksPerQueryFromStore(userID)), failedCounter), + limiter: NewLimiter(uint64(limits.MaxChunksPerQueryFromStore(userID)), failedCounter), } } } -func newSeriesLimiterFactory(limits *validation.Overrides, userID string) store.SeriesLimiterFactory { - return func(failedCounter prometheus.Counter) store.SeriesLimiter { +func newSeriesLimiterFactory(limits *validation.Overrides, userID string) SeriesLimiterFactory { + return func(failedCounter prometheus.Counter) SeriesLimiter { // Since limit overrides could be live reloaded, we have to get the current user's limit // each time a new limiter is instantiated. return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxFetchedSeriesPerQuery(userID)), failedCounter), + limiter: NewLimiter(uint64(limits.MaxFetchedSeriesPerQuery(userID)), failedCounter), } } } -func newBytesLimiterFactory(limits *validation.Overrides, userID string) store.BytesLimiterFactory { - return func(failedCounter prometheus.Counter) store.BytesLimiter { +func newBytesLimiterFactory(limits *validation.Overrides, userID string) BytesLimiterFactory { + return func(failedCounter prometheus.Counter) BytesLimiter { // Since limit overrides could be live reloaded, we have to get the current user's limit // each time a new limiter is instantiated. return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxDownloadedBytesPerRequest(userID)), failedCounter), + limiter: NewLimiter(uint64(limits.MaxDownloadedBytesPerRequest(userID)), failedCounter), } } } diff --git a/pkg/storegateway/bucket_stores_test.go b/pkg/storegateway/bucket_stores_test.go index 9d57d52d42..32bc8cf810 100644 --- a/pkg/storegateway/bucket_stores_test.go +++ b/pkg/storegateway/bucket_stores_test.go @@ -15,6 +15,8 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/go-kit/log" "github.com/gogo/status" "github.com/oklog/ulid" @@ -31,7 +33,6 @@ import ( thanos_metadata "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/logging" "go.uber.org/atomic" "google.golang.org/grpc/codes" @@ -120,7 +121,7 @@ func TestBucketStores_CustomerKeyError(t *testing.T) { require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), mBucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), mBucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) if tc.mockInitialSync { @@ -200,7 +201,7 @@ func TestBucketStores_InitialSync(t *testing.T) { require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) // Query series before the initial sync. @@ -276,7 +277,7 @@ func TestBucketStores_InitialSyncShouldRetryOnFailure(t *testing.T) { bucket = &failFirstGetBucket{Bucket: bucket} reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) // Initial sync should succeed even if a transient error occurs. @@ -336,7 +337,7 @@ func TestBucketStores_SyncBlocks(t *testing.T) { require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) // Run an initial sync to discover 1 block. @@ -420,12 +421,12 @@ func TestBucketStores_syncUsersBlocks(t *testing.T) { bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", allUsers, nil) - stores, err := NewBucketStores(cfg, testData.shardingStrategy, bucketClient, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + stores, err := NewBucketStores(cfg, testData.shardingStrategy, bucketClient, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) require.NoError(t, err) // Sync user stores and count the number of times the callback is called. var storesCount atomic.Int32 - err = stores.syncUsersBlocks(context.Background(), func(ctx context.Context, bs *store.BucketStore) error { + err = stores.syncUsersBlocks(context.Background(), func(ctx context.Context, bs *BucketStore) error { storesCount.Inc() return nil }) @@ -465,7 +466,7 @@ func testBucketStoresSeriesShouldCorrectlyQuerySeriesSpanningMultipleChunks(t *t require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) require.NoError(t, stores.InitialSync(ctx)) @@ -521,7 +522,7 @@ func TestBucketStores_Series_ShouldReturnErrorIfMaxInflightRequestIsReached(t *t bucket, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) require.NoError(t, err) - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) require.NoError(t, stores.InitialSync(context.Background())) @@ -542,7 +543,7 @@ func TestBucketStores_Series_ShouldNotCheckMaxInflightRequestsIfTheLimitIsDisabl bucket, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) require.NoError(t, err) - stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(), bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) require.NoError(t, stores.InitialSync(context.Background())) @@ -592,16 +593,16 @@ func generateStorageBlock(t *testing.T, storageDir, userID string, metricName st require.NoError(t, db.Snapshot(userDir, true)) } -func querySeries(stores *BucketStores, userID, metricName string, minT, maxT int64) ([]*storepb.Series, annotations.Annotations, error) { +func querySeries(stores *BucketStores, userID, metricName string, minT, maxT int64) ([]*typespb.Series, annotations.Annotations, error) { req := &storepb.SeriesRequest{ MinTime: minT, MaxTime: maxT, - Matchers: []storepb.LabelMatcher{{ - Type: storepb.LabelMatcher_EQ, + Matchers: []typespb.LabelMatcher{{ + Type: typespb.EQ, Name: labels.MetricName, Value: metricName, }}, - PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, + PartialResponseStrategy: typespb.ABORT, } ctx := setUserIDToGRPCContext(context.Background(), userID) @@ -615,12 +616,12 @@ func queryLabelsNames(stores *BucketStores, userID, metricName string, start, en req := &storepb.LabelNamesRequest{ Start: start, End: end, - Matchers: []storepb.LabelMatcher{{ - Type: storepb.LabelMatcher_EQ, + Matchers: []typespb.LabelMatcher{{ + Type: typespb.EQ, Name: labels.MetricName, Value: metricName, }}, - PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, + PartialResponseStrategy: typespb.ABORT, } ctx := setUserIDToGRPCContext(context.Background(), userID) @@ -632,12 +633,12 @@ func queryLabelsValues(stores *BucketStores, userID, labelName, metricName strin Start: start, End: end, Label: labelName, - Matchers: []storepb.LabelMatcher{{ - Type: storepb.LabelMatcher_EQ, + Matchers: []typespb.LabelMatcher{{ + Type: typespb.EQ, Name: labels.MetricName, Value: metricName, }}, - PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, + PartialResponseStrategy: typespb.ABORT, } ctx := setUserIDToGRPCContext(context.Background(), userID) @@ -686,7 +687,7 @@ func TestBucketStores_deleteLocalFilesForExcludedTenants(t *testing.T) { sharding := userShardingStrategy{} reg := prometheus.NewPedanticRegistry() - stores, err := NewBucketStores(cfg, &sharding, bucket, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + stores, err := NewBucketStores(cfg, &sharding, bucket, true, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) require.NoError(t, err) // Perform sync. diff --git a/pkg/storegateway/chunk_reader.go b/pkg/storegateway/chunk_reader.go new file mode 100644 index 0000000000..f4f284e0c7 --- /dev/null +++ b/pkg/storegateway/chunk_reader.go @@ -0,0 +1,303 @@ +package storegateway + +import ( + "bufio" + "context" + "encoding/binary" + "io" + "sort" + "sync" + "time" + + "github.com/alecthomas/units" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/thanos-io/thanos/pkg/runutil" + "github.com/weaveworks/common/httpgrpc" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc/codes" +) + +type loadIdx struct { + offset uint32 + // Indices, not actual entries and chunks. + seriesEntry int + chunk int +} + +type bucketChunkReader struct { + block *bucketBlock + + toLoad [][]loadIdx + + // chunkBytesMtx protects access to chunkBytes, when updated from chunks-loading goroutines. + // After chunks are loaded, mutex is no longer used. + chunkBytesMtx sync.Mutex + stats *queryStats + chunkBytes []*[]byte // Byte slice to return to the chunk pool on close. + + loadingChunksMtx sync.Mutex + loadingChunks bool + finishLoadingChks chan struct{} +} + +func newBucketChunkReader(block *bucketBlock) *bucketChunkReader { + return &bucketChunkReader{ + block: block, + stats: &queryStats{}, + toLoad: make([][]loadIdx, len(block.chunkObjs)), + } +} + +func (r *bucketChunkReader) reset() { + for i := range r.toLoad { + r.toLoad[i] = r.toLoad[i][:0] + } + r.loadingChunksMtx.Lock() + r.loadingChunks = false + r.finishLoadingChks = make(chan struct{}) + r.loadingChunksMtx.Unlock() +} + +func (r *bucketChunkReader) Close() error { + // NOTE(GiedriusS): we need to wait until loading chunks because loading + // chunks modifies r.block.chunkPool. + r.loadingChunksMtx.Lock() + loadingChks := r.loadingChunks + r.loadingChunksMtx.Unlock() + + if loadingChks { + <-r.finishLoadingChks + } + r.block.pendingReaders.Done() + + for _, b := range r.chunkBytes { + r.block.chunkPool.Put(b) + } + return nil +} + +// addLoad adds the chunk with id to the data set to be fetched. +// Chunk will be fetched and saved to refs[seriesEntry][chunk] upon r.load(refs, <...>) call. +func (r *bucketChunkReader) addLoad(id chunks.ChunkRef, seriesEntry, chunk int) error { + var ( + seq = int(id >> 32) + off = uint32(id) + ) + if seq >= len(r.toLoad) { + return errors.Errorf("reference sequence %d out of range", seq) + } + r.toLoad[seq] = append(r.toLoad[seq], loadIdx{off, seriesEntry, chunk}) + return nil +} + +// load loads all added chunks and saves resulting aggrs to refs. +func (r *bucketChunkReader) load(ctx context.Context, res []seriesEntry, aggrs []storepb.Aggr, calculateChunkChecksum bool, bytesLimiter BytesLimiter, tenant string) error { + r.loadingChunksMtx.Lock() + r.loadingChunks = true + r.loadingChunksMtx.Unlock() + + begin := time.Now() + defer func() { + r.stats.ChunksDownloadLatencySum += time.Since(begin) + + r.loadingChunksMtx.Lock() + r.loadingChunks = false + r.loadingChunksMtx.Unlock() + + close(r.finishLoadingChks) + }() + + g, ctx := errgroup.WithContext(ctx) + + for seq, pIdxs := range r.toLoad { + sort.Slice(pIdxs, func(i, j int) bool { + return pIdxs[i].offset < pIdxs[j].offset + }) + parts := r.block.partitioner.Partition(len(pIdxs), func(i int) (start, end uint64) { + return uint64(pIdxs[i].offset), uint64(pIdxs[i].offset) + uint64(r.block.estimatedMaxChunkSize) + }) + + for _, p := range parts { + if err := bytesLimiter.Reserve(uint64(p.End - p.Start)); err != nil { + return httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while fetching chunks: %s", err) + } + r.stats.DataDownloadedSizeSum += units.Base2Bytes(p.End - p.Start) + } + + for _, p := range parts { + seq := seq + p := p + indices := pIdxs[p.ElemRng[0]:p.ElemRng[1]] + g.Go(func() error { + return r.loadChunks(ctx, res, aggrs, seq, p, indices, calculateChunkChecksum, bytesLimiter, tenant) + }) + } + } + return g.Wait() +} + +// loadChunks will read range [start, end] from the segment file with sequence number seq. +// This data range covers chunks starting at supplied offsets. +func (r *bucketChunkReader) loadChunks(ctx context.Context, res []seriesEntry, aggrs []storepb.Aggr, seq int, part Part, pIdxs []loadIdx, calculateChunkChecksum bool, bytesLimiter BytesLimiter, tenant string) error { + fetchBegin := time.Now() + stats := new(queryStats) + defer func() { + stats.ChunksFetchDurationSum += time.Since(fetchBegin) + r.stats.merge(stats) + }() + + // Get a reader for the required range. + reader, err := r.block.chunkRangeReader(ctx, seq, int64(part.Start), int64(part.End-part.Start)) + if err != nil { + return errors.Wrap(err, "get range reader") + } + defer runutil.CloseWithLogOnErr(r.block.logger, reader, "readChunkRange close range reader") + bufReader := bufio.NewReaderSize(reader, r.block.estimatedMaxChunkSize) + + stats.chunksFetchCount++ + stats.chunksFetched += len(pIdxs) + stats.ChunksFetchedSizeSum += units.Base2Bytes(int(part.End - part.Start)) + + var ( + buf []byte + readOffset = int(pIdxs[0].offset) + + // Save a few allocations. + written int + diff uint32 + chunkLen int + n int + ) + + bufPooled, err := r.block.chunkPool.Get(r.block.estimatedMaxChunkSize) + if err == nil { + buf = *bufPooled + } else { + buf = make([]byte, r.block.estimatedMaxChunkSize) + } + defer r.block.chunkPool.Put(&buf) + + for i, pIdx := range pIdxs { + // Fast forward range reader to the next chunk start in case of sparse (for our purposes) byte range. + for readOffset < int(pIdx.offset) { + written, err = bufReader.Discard(int(pIdx.offset) - int(readOffset)) + if err != nil { + return errors.Wrap(err, "fast forward range reader") + } + readOffset += written + } + // Presume chunk length to be reasonably large for common use cases. + // However, declaration for EstimatedMaxChunkSize warns us some chunks could be larger in some rare cases. + // This is handled further down below. + chunkLen = r.block.estimatedMaxChunkSize + if i+1 < len(pIdxs) { + if diff = pIdxs[i+1].offset - pIdx.offset; int(diff) < chunkLen { + chunkLen = int(diff) + } + } + cb := buf[:chunkLen] + n, err = io.ReadFull(bufReader, cb) + readOffset += n + // Unexpected EOF for last chunk could be a valid case. Any other errors are definitely real. + if err != nil && !(errors.Is(err, io.ErrUnexpectedEOF) && i == len(pIdxs)-1) { + return errors.Wrapf(err, "read range for seq %d offset %x", seq, pIdx.offset) + } + + chunkDataLen, n := binary.Uvarint(cb) + if n < 1 { + return errors.New("reading chunk length failed") + } + + // Chunk length is n (number of bytes used to encode chunk data), 1 for chunk encoding and chunkDataLen for actual chunk data. + // There is also crc32 after the chunk, but we ignore that. + chunkLen = n + 1 + int(chunkDataLen) + if chunkLen <= len(cb) { + err = populateChunk(&(res[pIdx.seriesEntry].chks[pIdx.chunk]), rawChunk(cb[n:chunkLen]), aggrs, r.save, calculateChunkChecksum) + if err != nil { + return errors.Wrap(err, "populate chunk") + } + stats.chunksTouched++ + stats.ChunksTouchedSizeSum += units.Base2Bytes(int(chunkDataLen)) + continue + } + + r.block.metrics.chunkRefetches.WithLabelValues(tenant).Inc() + // If we didn't fetch enough data for the chunk, fetch more. + fetchBegin = time.Now() + // Read entire chunk into new buffer. + // TODO: readChunkRange call could be avoided for any chunk but last in this particular part. + if err := bytesLimiter.Reserve(uint64(chunkLen)); err != nil { + return httpgrpc.Errorf(int(codes.ResourceExhausted), "exceeded bytes limit while fetching chunks: %s", err) + } + stats.DataDownloadedSizeSum += units.Base2Bytes(chunkLen) + + nb, err := r.block.readChunkRange(ctx, seq, int64(pIdx.offset), int64(chunkLen), []byteRange{{offset: 0, length: chunkLen}}) + if err != nil { + return errors.Wrapf(err, "preloaded chunk too small, expecting %d, and failed to fetch full chunk", chunkLen) + } + if len(*nb) != chunkLen { + return errors.Errorf("preloaded chunk too small, expecting %d", chunkLen) + } + + stats.chunksFetchCount++ + stats.ChunksFetchedSizeSum += units.Base2Bytes(len(*nb)) + err = populateChunk(&(res[pIdx.seriesEntry].chks[pIdx.chunk]), rawChunk((*nb)[n:]), aggrs, r.save, calculateChunkChecksum) + if err != nil { + r.block.chunkPool.Put(nb) + return errors.Wrap(err, "populate chunk") + } + stats.chunksTouched++ + stats.ChunksTouchedSizeSum += units.Base2Bytes(int(chunkDataLen)) + + r.block.chunkPool.Put(nb) + } + return nil +} + +// save saves a copy of b's payload to a memory pool of its own and returns a new byte slice referencing said copy. +// Returned slice becomes invalid once r.block.chunkPool.Put() is called. +func (r *bucketChunkReader) save(b []byte) ([]byte, error) { + r.chunkBytesMtx.Lock() + defer r.chunkBytesMtx.Unlock() + // Ensure we never grow slab beyond original capacity. + if len(r.chunkBytes) == 0 || + cap(*r.chunkBytes[len(r.chunkBytes)-1])-len(*r.chunkBytes[len(r.chunkBytes)-1]) < len(b) { + s, err := r.block.chunkPool.Get(len(b)) + if err != nil { + return nil, errors.Wrap(err, "allocate chunk bytes") + } + r.chunkBytes = append(r.chunkBytes, s) + } + slab := r.chunkBytes[len(r.chunkBytes)-1] + *slab = append(*slab, b...) + return (*slab)[len(*slab)-len(b):], nil +} + +// rawChunk is a helper type that wraps a chunk's raw bytes and implements the chunkenc.Chunk +// interface over it. +// It is used to Store API responses which don't need to introspect and validate the chunk's contents. +type rawChunk []byte + +func (b rawChunk) Encoding() chunkenc.Encoding { + return chunkenc.Encoding(b[0]) +} + +func (b rawChunk) Bytes() []byte { + return b[1:] +} +func (b rawChunk) Compact() {} + +func (b rawChunk) Iterator(_ chunkenc.Iterator) chunkenc.Iterator { + panic("invalid call") +} + +func (b rawChunk) Appender() (chunkenc.Appender, error) { + panic("invalid call") +} + +func (b rawChunk) NumSamples() int { + panic("invalid call") +} diff --git a/pkg/storegateway/flushable.go b/pkg/storegateway/flushable.go new file mode 100644 index 0000000000..560e42ebe0 --- /dev/null +++ b/pkg/storegateway/flushable.go @@ -0,0 +1,84 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "github.com/prometheus/prometheus/model/labels" + "golang.org/x/exp/slices" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +type sortingStrategy uint64 + +const ( + sortingStrategyStore sortingStrategy = iota + 1 + sortingStrategyNone +) + +// flushableServer is an extension of storepb.Store_SeriesServer with a Flush method. +type flushableServer interface { + storepb.Store_SeriesServer + + Flush() error +} + +func newFlushableServer( + upstream storepb.Store_SeriesServer, + sortingsortingStrategy sortingStrategy, +) flushableServer { + switch sortingsortingStrategy { + case sortingStrategyStore: + return &resortingServer{Store_SeriesServer: upstream} + case sortingStrategyNone: + return &passthroughServer{Store_SeriesServer: upstream} + default: + // should not happen. + panic("unexpected sorting strategy") + } +} + +// passthroughServer is a flushableServer that forwards all data to +// an upstream server without additional processing. +type passthroughServer struct { + storepb.Store_SeriesServer +} + +func (p *passthroughServer) Flush() error { return nil } + +// resortingServer is a flushableServer that resorts all series by their labels. +// This is required if replica labels are stored internally in a TSDB. +// Data is resorted and sent to an upstream server upon calling Flush. +type resortingServer struct { + storepb.Store_SeriesServer + series []*typespb.Series +} + +func (r *resortingServer) Send(response *storepb.SeriesResponse) error { + if response.GetSeries() == nil { + return r.Store_SeriesServer.Send(response) + } + + series := response.GetSeries() + labelpb.ReAllocZLabelsStrings(&series.Labels, false) + r.series = append(r.series, series) + return nil +} + +func (r *resortingServer) Flush() error { + slices.SortFunc(r.series, func(a, b *typespb.Series) int { + return labels.Compare( + labelpb.ZLabelsToPromLabels(a.Labels), + labelpb.ZLabelsToPromLabels(b.Labels), + ) + }) + for _, response := range r.series { + if err := r.Store_SeriesServer.Send(storepb.NewSeriesResponse(response)); err != nil { + return err + } + } + return nil +} diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index 536a7f2556..74751b1a3b 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -14,7 +14,6 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/thanos-io/objstore" "github.com/thanos-io/thanos/pkg/extprom" - "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/logging" "github.com/cortexproject/cortex/pkg/ring" @@ -22,6 +21,7 @@ import ( "github.com/cortexproject/cortex/pkg/storage/bucket" cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -202,7 +202,7 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf shardingStrategy = NewNoShardingStrategy() } - g.stores, err = NewBucketStores(storageCfg, shardingStrategy, bucketClient, limits, logLevel, logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "store-gateway"}, reg)) + g.stores, err = NewBucketStores(storageCfg, shardingStrategy, bucketClient, true, limits, logLevel, logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "store-gateway"}, reg)) if err != nil { return nil, errors.Wrap(err, "create bucket stores") } @@ -362,6 +362,10 @@ func (g *StoreGateway) Series(req *storepb.SeriesRequest, srv storegatewaypb.Sto return g.stores.Series(req, srv) } +func (g *StoreGateway) Select(req *storepb.SelectRequest, srv storegatewaypb.StoreGateway_SelectServer) error { + return g.stores.Select(req, srv) +} + // LabelNames implements the Storegateway proto service. func (g *StoreGateway) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { return g.stores.LabelNames(ctx, req) diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index b499f72aa3..d5430c025f 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -15,6 +15,8 @@ import ( "google.golang.org/grpc/codes" + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/pkg/errors" @@ -29,7 +31,6 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/store/labelpb" - "github.com/thanos-io/thanos/pkg/store/storepb" "google.golang.org/grpc/status" "github.com/cortexproject/cortex/pkg/ring" @@ -960,8 +961,8 @@ func TestStoreGateway_SeriesQueryingShouldRemoveExternalLabels(t *testing.T) { req := &storepb.SeriesRequest{ MinTime: minT, MaxTime: maxT, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, + Matchers: []typespb.LabelMatcher{ + {Type: typespb.RE, Name: "__name__", Value: ".*"}, }, } @@ -1031,8 +1032,8 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxChunksPerQueryLimit(t *testi req := &storepb.SeriesRequest{ MinTime: minT, MaxTime: maxT, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, + Matchers: []typespb.LabelMatcher{ + {Type: typespb.RE, Name: "__name__", Value: ".*"}, }, } @@ -1120,8 +1121,8 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxSeriesPerQueryLimit(t *testi req := &storepb.SeriesRequest{ MinTime: minT, MaxTime: maxT, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, + Matchers: []typespb.LabelMatcher{ + {Type: typespb.RE, Name: "__name__", Value: ".*"}, }, } @@ -1236,7 +1237,7 @@ func mockTSDB(t *testing.T, dir string, numSeries, numBlocks int, minT, maxT int require.NoError(t, db.Close()) } -func readSamplesFromChunks(rawChunks []storepb.AggrChunk) ([]sample, error) { +func readSamplesFromChunks(rawChunks []typespb.AggrChunk) ([]sample, error) { var samples []sample for _, rawChunk := range rawChunks { diff --git a/pkg/storegateway/hintspb/custom.go b/pkg/storegateway/hintspb/custom.go new file mode 100644 index 0000000000..bf82d245e2 --- /dev/null +++ b/pkg/storegateway/hintspb/custom.go @@ -0,0 +1,53 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package hintspb + +import "github.com/oklog/ulid" + +func (m *SeriesResponseHints) AddQueriedBlock(id ulid.ULID) { + m.QueriedBlocks = append(m.QueriedBlocks, Block{ + Id: id.String(), + }) +} + +func (m *LabelNamesResponseHints) AddQueriedBlock(id ulid.ULID) { + m.QueriedBlocks = append(m.QueriedBlocks, Block{ + Id: id.String(), + }) +} + +func (m *LabelValuesResponseHints) AddQueriedBlock(id ulid.ULID) { + m.QueriedBlocks = append(m.QueriedBlocks, Block{ + Id: id.String(), + }) +} + +func (m *QueryStats) Merge(other *QueryStats) { + m.BlocksQueried += other.BlocksQueried + m.MergedSeriesCount += other.MergedSeriesCount + m.MergedChunksCount += other.MergedChunksCount + m.DataDownloadedSizeSum += other.DataDownloadedSizeSum + + m.PostingsFetched += other.PostingsFetched + m.PostingsToFetch += other.PostingsToFetch + m.PostingsFetchCount += other.PostingsFetchCount + m.PostingsFetchedSizeSum += other.PostingsFetchedSizeSum + m.PostingsTouched += other.PostingsTouched + m.PostingsTouchedSizeSum += other.PostingsTouchedSizeSum + + m.SeriesFetched += other.SeriesFetched + m.SeriesFetchCount += other.SeriesFetchCount + m.SeriesFetchedSizeSum += other.SeriesFetchedSizeSum + m.SeriesTouched += other.SeriesTouched + m.SeriesTouchedSizeSum += other.SeriesTouchedSizeSum + + m.ChunksFetched += other.ChunksFetched + m.ChunksFetchCount += other.ChunksFetchCount + m.ChunksFetchedSizeSum += other.ChunksFetchedSizeSum + m.ChunksTouched += other.ChunksTouched + m.ChunksTouchedSizeSum += other.ChunksTouchedSizeSum + + m.GetAllDuration += other.GetAllDuration + m.MergeDuration += other.MergeDuration +} diff --git a/pkg/storegateway/hintspb/hints.pb.go b/pkg/storegateway/hintspb/hints.pb.go new file mode 100644 index 0000000000..0345e2260b --- /dev/null +++ b/pkg/storegateway/hintspb/hints.pb.go @@ -0,0 +1,2904 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: hints.proto + +package hintspb + +import ( + fmt "fmt" + typespb "github.com/cortexproject/cortex/pkg/storegateway/typespb" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" + _ "github.com/golang/protobuf/ptypes/duration" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strings "strings" + time "time" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf +var _ = time.Kitchen + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type SeriesRequestHints struct { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + BlockMatchers []typespb.LabelMatcher `protobuf:"bytes,1,rep,name=block_matchers,json=blockMatchers,proto3" json:"block_matchers"` + EnableQueryStats bool `protobuf:"varint,2,opt,name=enable_query_stats,json=enableQueryStats,proto3" json:"enable_query_stats,omitempty"` +} + +func (m *SeriesRequestHints) Reset() { *m = SeriesRequestHints{} } +func (*SeriesRequestHints) ProtoMessage() {} +func (*SeriesRequestHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{0} +} +func (m *SeriesRequestHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesRequestHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesRequestHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesRequestHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesRequestHints.Merge(m, src) +} +func (m *SeriesRequestHints) XXX_Size() int { + return m.Size() +} +func (m *SeriesRequestHints) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesRequestHints.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesRequestHints proto.InternalMessageInfo + +type SeriesResponseHints struct { + /// queried_blocks is the list of blocks that have been queried. + QueriedBlocks []Block `protobuf:"bytes,1,rep,name=queried_blocks,json=queriedBlocks,proto3" json:"queried_blocks"` + /// query_stats contains statistics of querying store gateway. + QueryStats *QueryStats `protobuf:"bytes,2,opt,name=query_stats,json=queryStats,proto3" json:"query_stats,omitempty"` +} + +func (m *SeriesResponseHints) Reset() { *m = SeriesResponseHints{} } +func (*SeriesResponseHints) ProtoMessage() {} +func (*SeriesResponseHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{1} +} +func (m *SeriesResponseHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesResponseHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesResponseHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesResponseHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesResponseHints.Merge(m, src) +} +func (m *SeriesResponseHints) XXX_Size() int { + return m.Size() +} +func (m *SeriesResponseHints) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesResponseHints.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesResponseHints proto.InternalMessageInfo + +type Block struct { + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` +} + +func (m *Block) Reset() { *m = Block{} } +func (*Block) ProtoMessage() {} +func (*Block) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{2} +} +func (m *Block) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Block) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Block.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Block) XXX_Merge(src proto.Message) { + xxx_messageInfo_Block.Merge(m, src) +} +func (m *Block) XXX_Size() int { + return m.Size() +} +func (m *Block) XXX_DiscardUnknown() { + xxx_messageInfo_Block.DiscardUnknown(m) +} + +var xxx_messageInfo_Block proto.InternalMessageInfo + +type LabelNamesRequestHints struct { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + BlockMatchers []typespb.LabelMatcher `protobuf:"bytes,1,rep,name=block_matchers,json=blockMatchers,proto3" json:"block_matchers"` +} + +func (m *LabelNamesRequestHints) Reset() { *m = LabelNamesRequestHints{} } +func (*LabelNamesRequestHints) ProtoMessage() {} +func (*LabelNamesRequestHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{3} +} +func (m *LabelNamesRequestHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelNamesRequestHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelNamesRequestHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelNamesRequestHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelNamesRequestHints.Merge(m, src) +} +func (m *LabelNamesRequestHints) XXX_Size() int { + return m.Size() +} +func (m *LabelNamesRequestHints) XXX_DiscardUnknown() { + xxx_messageInfo_LabelNamesRequestHints.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelNamesRequestHints proto.InternalMessageInfo + +type LabelNamesResponseHints struct { + /// queried_blocks is the list of blocks that have been queried. + QueriedBlocks []Block `protobuf:"bytes,1,rep,name=queried_blocks,json=queriedBlocks,proto3" json:"queried_blocks"` +} + +func (m *LabelNamesResponseHints) Reset() { *m = LabelNamesResponseHints{} } +func (*LabelNamesResponseHints) ProtoMessage() {} +func (*LabelNamesResponseHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{4} +} +func (m *LabelNamesResponseHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelNamesResponseHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelNamesResponseHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelNamesResponseHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelNamesResponseHints.Merge(m, src) +} +func (m *LabelNamesResponseHints) XXX_Size() int { + return m.Size() +} +func (m *LabelNamesResponseHints) XXX_DiscardUnknown() { + xxx_messageInfo_LabelNamesResponseHints.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelNamesResponseHints proto.InternalMessageInfo + +type LabelValuesRequestHints struct { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + BlockMatchers []typespb.LabelMatcher `protobuf:"bytes,1,rep,name=block_matchers,json=blockMatchers,proto3" json:"block_matchers"` +} + +func (m *LabelValuesRequestHints) Reset() { *m = LabelValuesRequestHints{} } +func (*LabelValuesRequestHints) ProtoMessage() {} +func (*LabelValuesRequestHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{5} +} +func (m *LabelValuesRequestHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelValuesRequestHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelValuesRequestHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelValuesRequestHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelValuesRequestHints.Merge(m, src) +} +func (m *LabelValuesRequestHints) XXX_Size() int { + return m.Size() +} +func (m *LabelValuesRequestHints) XXX_DiscardUnknown() { + xxx_messageInfo_LabelValuesRequestHints.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelValuesRequestHints proto.InternalMessageInfo + +type LabelValuesResponseHints struct { + /// queried_blocks is the list of blocks that have been queried. + QueriedBlocks []Block `protobuf:"bytes,1,rep,name=queried_blocks,json=queriedBlocks,proto3" json:"queried_blocks"` +} + +func (m *LabelValuesResponseHints) Reset() { *m = LabelValuesResponseHints{} } +func (*LabelValuesResponseHints) ProtoMessage() {} +func (*LabelValuesResponseHints) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{6} +} +func (m *LabelValuesResponseHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelValuesResponseHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelValuesResponseHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelValuesResponseHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelValuesResponseHints.Merge(m, src) +} +func (m *LabelValuesResponseHints) XXX_Size() int { + return m.Size() +} +func (m *LabelValuesResponseHints) XXX_DiscardUnknown() { + xxx_messageInfo_LabelValuesResponseHints.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelValuesResponseHints proto.InternalMessageInfo + +// / QueryStats fields are unstable and might change in the future. +type QueryStats struct { + BlocksQueried int64 `protobuf:"varint,1,opt,name=blocks_queried,json=blocksQueried,proto3" json:"blocks_queried,omitempty"` + MergedSeriesCount int64 `protobuf:"varint,2,opt,name=merged_series_count,json=mergedSeriesCount,proto3" json:"merged_series_count,omitempty"` + MergedChunksCount int64 `protobuf:"varint,3,opt,name=merged_chunks_count,json=mergedChunksCount,proto3" json:"merged_chunks_count,omitempty"` + PostingsTouched int64 `protobuf:"varint,4,opt,name=postings_touched,json=postingsTouched,proto3" json:"postings_touched,omitempty"` + PostingsTouchedSizeSum int64 `protobuf:"varint,5,opt,name=postings_touched_size_sum,json=postingsTouchedSizeSum,proto3" json:"postings_touched_size_sum,omitempty"` + PostingsToFetch int64 `protobuf:"varint,6,opt,name=postings_to_fetch,json=postingsToFetch,proto3" json:"postings_to_fetch,omitempty"` + PostingsFetched int64 `protobuf:"varint,7,opt,name=postings_fetched,json=postingsFetched,proto3" json:"postings_fetched,omitempty"` + PostingsFetchedSizeSum int64 `protobuf:"varint,8,opt,name=postings_fetched_size_sum,json=postingsFetchedSizeSum,proto3" json:"postings_fetched_size_sum,omitempty"` + PostingsFetchCount int64 `protobuf:"varint,9,opt,name=postings_fetch_count,json=postingsFetchCount,proto3" json:"postings_fetch_count,omitempty"` + SeriesTouched int64 `protobuf:"varint,10,opt,name=series_touched,json=seriesTouched,proto3" json:"series_touched,omitempty"` + SeriesTouchedSizeSum int64 `protobuf:"varint,11,opt,name=series_touched_size_sum,json=seriesTouchedSizeSum,proto3" json:"series_touched_size_sum,omitempty"` + SeriesFetched int64 `protobuf:"varint,12,opt,name=series_fetched,json=seriesFetched,proto3" json:"series_fetched,omitempty"` + SeriesFetchedSizeSum int64 `protobuf:"varint,13,opt,name=series_fetched_size_sum,json=seriesFetchedSizeSum,proto3" json:"series_fetched_size_sum,omitempty"` + SeriesFetchCount int64 `protobuf:"varint,14,opt,name=series_fetch_count,json=seriesFetchCount,proto3" json:"series_fetch_count,omitempty"` + ChunksTouched int64 `protobuf:"varint,15,opt,name=chunks_touched,json=chunksTouched,proto3" json:"chunks_touched,omitempty"` + ChunksTouchedSizeSum int64 `protobuf:"varint,16,opt,name=chunks_touched_size_sum,json=chunksTouchedSizeSum,proto3" json:"chunks_touched_size_sum,omitempty"` + ChunksFetched int64 `protobuf:"varint,17,opt,name=chunks_fetched,json=chunksFetched,proto3" json:"chunks_fetched,omitempty"` + ChunksFetchedSizeSum int64 `protobuf:"varint,18,opt,name=chunks_fetched_size_sum,json=chunksFetchedSizeSum,proto3" json:"chunks_fetched_size_sum,omitempty"` + ChunksFetchCount int64 `protobuf:"varint,19,opt,name=chunks_fetch_count,json=chunksFetchCount,proto3" json:"chunks_fetch_count,omitempty"` + DataDownloadedSizeSum int64 `protobuf:"varint,20,opt,name=data_downloaded_size_sum,json=dataDownloadedSizeSum,proto3" json:"data_downloaded_size_sum,omitempty"` + GetAllDuration time.Duration `protobuf:"bytes,21,opt,name=get_all_duration,json=getAllDuration,proto3,stdduration" json:"get_all_duration"` + MergeDuration time.Duration `protobuf:"bytes,22,opt,name=merge_duration,json=mergeDuration,proto3,stdduration" json:"merge_duration"` +} + +func (m *QueryStats) Reset() { *m = QueryStats{} } +func (*QueryStats) ProtoMessage() {} +func (*QueryStats) Descriptor() ([]byte, []int) { + return fileDescriptor_522be8e0d2634375, []int{7} +} +func (m *QueryStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_QueryStats.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *QueryStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryStats.Merge(m, src) +} +func (m *QueryStats) XXX_Size() int { + return m.Size() +} +func (m *QueryStats) XXX_DiscardUnknown() { + xxx_messageInfo_QueryStats.DiscardUnknown(m) +} + +var xxx_messageInfo_QueryStats proto.InternalMessageInfo + +func init() { + proto.RegisterType((*SeriesRequestHints)(nil), "hintspb.SeriesRequestHints") + proto.RegisterType((*SeriesResponseHints)(nil), "hintspb.SeriesResponseHints") + proto.RegisterType((*Block)(nil), "hintspb.Block") + proto.RegisterType((*LabelNamesRequestHints)(nil), "hintspb.LabelNamesRequestHints") + proto.RegisterType((*LabelNamesResponseHints)(nil), "hintspb.LabelNamesResponseHints") + proto.RegisterType((*LabelValuesRequestHints)(nil), "hintspb.LabelValuesRequestHints") + proto.RegisterType((*LabelValuesResponseHints)(nil), "hintspb.LabelValuesResponseHints") + proto.RegisterType((*QueryStats)(nil), "hintspb.QueryStats") +} + +func init() { proto.RegisterFile("hints.proto", fileDescriptor_522be8e0d2634375) } + +var fileDescriptor_522be8e0d2634375 = []byte{ + // 792 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4d, 0x4f, 0xdb, 0x48, + 0x18, 0xb6, 0x09, 0x1f, 0x61, 0xb2, 0x98, 0xe0, 0x04, 0x30, 0x1c, 0x0c, 0x8a, 0x84, 0xc4, 0xae, + 0x90, 0xb3, 0x62, 0xb5, 0x5a, 0xed, 0xee, 0x89, 0x80, 0xd0, 0x6a, 0xb5, 0xac, 0x44, 0x52, 0x51, + 0xa9, 0x6a, 0x65, 0xf9, 0x63, 0x70, 0x5c, 0x1c, 0x8f, 0xf1, 0x8c, 0x45, 0xe1, 0xd4, 0x53, 0x8f, + 0x55, 0x8f, 0xfd, 0x09, 0xfd, 0x29, 0x1c, 0x39, 0x72, 0x6a, 0x9b, 0xa0, 0x4a, 0x3d, 0xf2, 0x13, + 0x2a, 0xcf, 0x47, 0x3c, 0x0e, 0x97, 0x1e, 0xe8, 0x09, 0xe6, 0x7d, 0x9f, 0xe7, 0x99, 0xe7, 0x79, + 0x67, 0xe2, 0x01, 0xb5, 0x7e, 0x18, 0x13, 0x6c, 0x25, 0x29, 0x22, 0x48, 0x9f, 0xa3, 0x8b, 0xc4, + 0x5d, 0x6f, 0x06, 0x28, 0x40, 0xb4, 0xd6, 0xce, 0xff, 0x63, 0xed, 0x75, 0x33, 0x40, 0x28, 0x88, + 0x60, 0x9b, 0xae, 0xdc, 0xec, 0xb4, 0xed, 0x67, 0xa9, 0x43, 0x42, 0x14, 0xf3, 0xfe, 0x41, 0x10, + 0x92, 0x7e, 0xe6, 0x5a, 0x1e, 0x1a, 0xb4, 0x3d, 0x94, 0x12, 0xf8, 0x2a, 0x49, 0xd1, 0x4b, 0xe8, + 0x11, 0xbe, 0x6a, 0x27, 0x67, 0x41, 0x1b, 0x13, 0x94, 0xc2, 0xc0, 0x21, 0xf0, 0xc2, 0xb9, 0x6c, + 0x93, 0xcb, 0x04, 0xe2, 0xc4, 0x65, 0x7f, 0x99, 0x4a, 0xeb, 0x8d, 0x0a, 0xf4, 0x1e, 0x4c, 0x43, + 0x88, 0xbb, 0xf0, 0x3c, 0x83, 0x98, 0xfc, 0x93, 0x9b, 0xd2, 0x3b, 0x40, 0x73, 0x23, 0xe4, 0x9d, + 0xd9, 0x03, 0x87, 0x78, 0x7d, 0x98, 0x62, 0x43, 0xdd, 0xac, 0x6c, 0xd7, 0x76, 0x97, 0x2d, 0x2e, + 0x62, 0xfd, 0xe7, 0xb8, 0x30, 0x3a, 0x62, 0xdd, 0xce, 0xf4, 0xf5, 0xc7, 0x0d, 0xa5, 0xbb, 0x40, + 0x29, 0xbc, 0x86, 0xf5, 0x1d, 0xa0, 0xc3, 0xd8, 0x71, 0x23, 0x68, 0x9f, 0x67, 0x30, 0xbd, 0xb4, + 0x31, 0x71, 0x08, 0x36, 0xa6, 0x36, 0xd5, 0xed, 0x6a, 0xb7, 0xce, 0x3a, 0xc7, 0x79, 0xa3, 0x97, + 0xd7, 0x5b, 0x6f, 0x55, 0xd0, 0x10, 0x46, 0x70, 0x82, 0x62, 0x0c, 0x99, 0x93, 0xbf, 0x81, 0x96, + 0xd3, 0x43, 0xe8, 0xdb, 0x54, 0x5e, 0x38, 0xd1, 0x2c, 0x3e, 0x3e, 0xab, 0x93, 0x97, 0x85, 0x05, + 0x8e, 0xa5, 0x35, 0xac, 0xff, 0x05, 0x6a, 0x93, 0x7b, 0xd7, 0x76, 0x1b, 0x63, 0x66, 0xb1, 0x3d, + 0xa5, 0xab, 0x5d, 0x70, 0x5e, 0x18, 0x5a, 0x05, 0x33, 0x54, 0x45, 0xd7, 0xc0, 0x54, 0xe8, 0x1b, + 0xea, 0xa6, 0xba, 0x3d, 0xdf, 0x9d, 0x0a, 0xfd, 0xd6, 0x73, 0xb0, 0x42, 0xc3, 0xff, 0xef, 0x0c, + 0x1e, 0x7f, 0x6a, 0xad, 0x13, 0xb0, 0x2a, 0xab, 0x3f, 0xd6, 0x28, 0x5a, 0x2f, 0xb8, 0xee, 0x89, + 0x13, 0x65, 0x3f, 0xc0, 0xf6, 0x53, 0x60, 0x94, 0xe4, 0x1f, 0xcd, 0xf7, 0x97, 0x2a, 0x00, 0xc5, + 0x39, 0xe9, 0x5b, 0xdc, 0x2b, 0xb6, 0x39, 0x8c, 0x1e, 0x4c, 0x85, 0xdb, 0xc1, 0xc7, 0xac, 0xa8, + 0x5b, 0xa0, 0x31, 0x80, 0x69, 0x00, 0x7d, 0x1b, 0xd3, 0x3b, 0x65, 0x7b, 0x28, 0x8b, 0x09, 0xbd, + 0x00, 0x95, 0xee, 0x12, 0x6b, 0xb1, 0xdb, 0xb6, 0x9f, 0x37, 0x24, 0xbc, 0xd7, 0xcf, 0xe2, 0x33, + 0x81, 0xaf, 0xc8, 0xf8, 0x7d, 0xda, 0x61, 0xf8, 0x9f, 0x41, 0x3d, 0x41, 0x98, 0x84, 0x71, 0x80, + 0x6d, 0x82, 0x32, 0xaf, 0x0f, 0x7d, 0x63, 0x9a, 0x82, 0x17, 0x45, 0xfd, 0x09, 0x2b, 0xeb, 0x7f, + 0x82, 0xb5, 0x49, 0xa8, 0x8d, 0xc3, 0x2b, 0x68, 0xe3, 0x6c, 0x60, 0xcc, 0x50, 0xce, 0xca, 0x04, + 0xa7, 0x17, 0x5e, 0xc1, 0x5e, 0x36, 0xd0, 0x7f, 0x01, 0x4b, 0x12, 0xd5, 0x3e, 0x85, 0xc4, 0xeb, + 0x1b, 0xb3, 0x93, 0xdb, 0x1c, 0xe6, 0xe5, 0x92, 0x23, 0x0a, 0x84, 0xbe, 0x31, 0x57, 0x86, 0x1e, + 0xb2, 0x72, 0xc9, 0x11, 0x87, 0x16, 0x8e, 0xaa, 0x65, 0x47, 0x9c, 0x23, 0x1c, 0xfd, 0x0a, 0x9a, + 0x65, 0x2a, 0x1f, 0xd4, 0x3c, 0x65, 0xe9, 0x25, 0x16, 0x9b, 0xd4, 0x16, 0xd0, 0xf8, 0x11, 0x88, + 0x39, 0x01, 0x76, 0x60, 0xac, 0x2a, 0xa6, 0xf4, 0x3b, 0x58, 0x2d, 0xc3, 0x0a, 0x47, 0x35, 0x8a, + 0x6f, 0x96, 0xf0, 0xc2, 0x4f, 0xa1, 0x2e, 0x32, 0xff, 0x24, 0xab, 0x8b, 0xc4, 0x85, 0xfa, 0x83, + 0xbc, 0x0b, 0xb2, 0xfa, 0x44, 0xda, 0x1d, 0xa0, 0xcb, 0x34, 0x9e, 0x55, 0xa3, 0x8c, 0xba, 0xc4, + 0x18, 0x27, 0xe5, 0x97, 0x47, 0x24, 0x5d, 0x64, 0x5e, 0x58, 0x55, 0x4a, 0x5a, 0x86, 0x15, 0x5e, + 0xea, 0xcc, 0x4b, 0x09, 0x2f, 0x25, 0xe5, 0x34, 0x91, 0x74, 0x49, 0x56, 0x97, 0x92, 0x96, 0x61, + 0x85, 0xba, 0x2e, 0xab, 0x3f, 0x4c, 0x2a, 0xd3, 0x78, 0xd2, 0x06, 0x4b, 0x2a, 0x31, 0x58, 0xd2, + 0x3f, 0x80, 0xe1, 0x3b, 0xc4, 0xb1, 0x7d, 0x74, 0x11, 0x47, 0xc8, 0xf1, 0xe5, 0x5d, 0x9a, 0x94, + 0xb3, 0x9c, 0xf7, 0x0f, 0xc6, 0x6d, 0xb1, 0xcd, 0x11, 0xa8, 0x07, 0x90, 0xd8, 0x4e, 0x14, 0xd9, + 0xe2, 0x35, 0x33, 0x96, 0xe9, 0x47, 0x79, 0xcd, 0x62, 0xcf, 0x9d, 0x25, 0x9e, 0x3b, 0xeb, 0x80, + 0x03, 0x3a, 0xd5, 0xfc, 0xb3, 0xf0, 0xfe, 0xd3, 0x86, 0xda, 0xd5, 0x02, 0x48, 0xf6, 0xa2, 0x48, + 0x74, 0xf4, 0x7f, 0x81, 0x46, 0x7f, 0x9a, 0x85, 0xd8, 0xca, 0xf7, 0x8b, 0x2d, 0x50, 0xea, 0xb8, + 0xb1, 0x77, 0x3d, 0x34, 0x95, 0x9b, 0xa1, 0xa9, 0xdc, 0x0e, 0x4d, 0xe5, 0x7e, 0x68, 0xaa, 0xaf, + 0x47, 0xa6, 0xfa, 0x61, 0x64, 0xaa, 0xd7, 0x23, 0x53, 0xbd, 0x19, 0x99, 0xea, 0xe7, 0x91, 0xa9, + 0x7e, 0x1d, 0x99, 0xca, 0xfd, 0xc8, 0x54, 0xdf, 0xdd, 0x99, 0xca, 0xcd, 0x9d, 0xa9, 0xdc, 0xde, + 0x99, 0xca, 0x33, 0xf1, 0x8e, 0xbb, 0xb3, 0x74, 0xbb, 0xdf, 0xbe, 0x05, 0x00, 0x00, 0xff, 0xff, + 0x7f, 0x6c, 0x33, 0xac, 0xe6, 0x07, 0x00, 0x00, +} + +func (this *SeriesRequestHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesRequestHints) + if !ok { + that2, ok := that.(SeriesRequestHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.BlockMatchers) != len(that1.BlockMatchers) { + return false + } + for i := range this.BlockMatchers { + if !this.BlockMatchers[i].Equal(&that1.BlockMatchers[i]) { + return false + } + } + if this.EnableQueryStats != that1.EnableQueryStats { + return false + } + return true +} +func (this *SeriesResponseHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponseHints) + if !ok { + that2, ok := that.(SeriesResponseHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.QueriedBlocks) != len(that1.QueriedBlocks) { + return false + } + for i := range this.QueriedBlocks { + if !this.QueriedBlocks[i].Equal(&that1.QueriedBlocks[i]) { + return false + } + } + if !this.QueryStats.Equal(that1.QueryStats) { + return false + } + return true +} +func (this *Block) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Block) + if !ok { + that2, ok := that.(Block) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Id != that1.Id { + return false + } + return true +} +func (this *LabelNamesRequestHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelNamesRequestHints) + if !ok { + that2, ok := that.(LabelNamesRequestHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.BlockMatchers) != len(that1.BlockMatchers) { + return false + } + for i := range this.BlockMatchers { + if !this.BlockMatchers[i].Equal(&that1.BlockMatchers[i]) { + return false + } + } + return true +} +func (this *LabelNamesResponseHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelNamesResponseHints) + if !ok { + that2, ok := that.(LabelNamesResponseHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.QueriedBlocks) != len(that1.QueriedBlocks) { + return false + } + for i := range this.QueriedBlocks { + if !this.QueriedBlocks[i].Equal(&that1.QueriedBlocks[i]) { + return false + } + } + return true +} +func (this *LabelValuesRequestHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelValuesRequestHints) + if !ok { + that2, ok := that.(LabelValuesRequestHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.BlockMatchers) != len(that1.BlockMatchers) { + return false + } + for i := range this.BlockMatchers { + if !this.BlockMatchers[i].Equal(&that1.BlockMatchers[i]) { + return false + } + } + return true +} +func (this *LabelValuesResponseHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelValuesResponseHints) + if !ok { + that2, ok := that.(LabelValuesResponseHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.QueriedBlocks) != len(that1.QueriedBlocks) { + return false + } + for i := range this.QueriedBlocks { + if !this.QueriedBlocks[i].Equal(&that1.QueriedBlocks[i]) { + return false + } + } + return true +} +func (this *QueryStats) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryStats) + if !ok { + that2, ok := that.(QueryStats) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BlocksQueried != that1.BlocksQueried { + return false + } + if this.MergedSeriesCount != that1.MergedSeriesCount { + return false + } + if this.MergedChunksCount != that1.MergedChunksCount { + return false + } + if this.PostingsTouched != that1.PostingsTouched { + return false + } + if this.PostingsTouchedSizeSum != that1.PostingsTouchedSizeSum { + return false + } + if this.PostingsToFetch != that1.PostingsToFetch { + return false + } + if this.PostingsFetched != that1.PostingsFetched { + return false + } + if this.PostingsFetchedSizeSum != that1.PostingsFetchedSizeSum { + return false + } + if this.PostingsFetchCount != that1.PostingsFetchCount { + return false + } + if this.SeriesTouched != that1.SeriesTouched { + return false + } + if this.SeriesTouchedSizeSum != that1.SeriesTouchedSizeSum { + return false + } + if this.SeriesFetched != that1.SeriesFetched { + return false + } + if this.SeriesFetchedSizeSum != that1.SeriesFetchedSizeSum { + return false + } + if this.SeriesFetchCount != that1.SeriesFetchCount { + return false + } + if this.ChunksTouched != that1.ChunksTouched { + return false + } + if this.ChunksTouchedSizeSum != that1.ChunksTouchedSizeSum { + return false + } + if this.ChunksFetched != that1.ChunksFetched { + return false + } + if this.ChunksFetchedSizeSum != that1.ChunksFetchedSizeSum { + return false + } + if this.ChunksFetchCount != that1.ChunksFetchCount { + return false + } + if this.DataDownloadedSizeSum != that1.DataDownloadedSizeSum { + return false + } + if this.GetAllDuration != that1.GetAllDuration { + return false + } + if this.MergeDuration != that1.MergeDuration { + return false + } + return true +} +func (this *SeriesRequestHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&hintspb.SeriesRequestHints{") + if this.BlockMatchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.BlockMatchers)) + for i := range vs { + vs[i] = &this.BlockMatchers[i] + } + s = append(s, "BlockMatchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "EnableQueryStats: "+fmt.Sprintf("%#v", this.EnableQueryStats)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SeriesResponseHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&hintspb.SeriesResponseHints{") + if this.QueriedBlocks != nil { + vs := make([]*Block, len(this.QueriedBlocks)) + for i := range vs { + vs[i] = &this.QueriedBlocks[i] + } + s = append(s, "QueriedBlocks: "+fmt.Sprintf("%#v", vs)+",\n") + } + if this.QueryStats != nil { + s = append(s, "QueryStats: "+fmt.Sprintf("%#v", this.QueryStats)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Block) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&hintspb.Block{") + s = append(s, "Id: "+fmt.Sprintf("%#v", this.Id)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelNamesRequestHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&hintspb.LabelNamesRequestHints{") + if this.BlockMatchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.BlockMatchers)) + for i := range vs { + vs[i] = &this.BlockMatchers[i] + } + s = append(s, "BlockMatchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelNamesResponseHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&hintspb.LabelNamesResponseHints{") + if this.QueriedBlocks != nil { + vs := make([]*Block, len(this.QueriedBlocks)) + for i := range vs { + vs[i] = &this.QueriedBlocks[i] + } + s = append(s, "QueriedBlocks: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelValuesRequestHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&hintspb.LabelValuesRequestHints{") + if this.BlockMatchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.BlockMatchers)) + for i := range vs { + vs[i] = &this.BlockMatchers[i] + } + s = append(s, "BlockMatchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelValuesResponseHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&hintspb.LabelValuesResponseHints{") + if this.QueriedBlocks != nil { + vs := make([]*Block, len(this.QueriedBlocks)) + for i := range vs { + vs[i] = &this.QueriedBlocks[i] + } + s = append(s, "QueriedBlocks: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *QueryStats) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 26) + s = append(s, "&hintspb.QueryStats{") + s = append(s, "BlocksQueried: "+fmt.Sprintf("%#v", this.BlocksQueried)+",\n") + s = append(s, "MergedSeriesCount: "+fmt.Sprintf("%#v", this.MergedSeriesCount)+",\n") + s = append(s, "MergedChunksCount: "+fmt.Sprintf("%#v", this.MergedChunksCount)+",\n") + s = append(s, "PostingsTouched: "+fmt.Sprintf("%#v", this.PostingsTouched)+",\n") + s = append(s, "PostingsTouchedSizeSum: "+fmt.Sprintf("%#v", this.PostingsTouchedSizeSum)+",\n") + s = append(s, "PostingsToFetch: "+fmt.Sprintf("%#v", this.PostingsToFetch)+",\n") + s = append(s, "PostingsFetched: "+fmt.Sprintf("%#v", this.PostingsFetched)+",\n") + s = append(s, "PostingsFetchedSizeSum: "+fmt.Sprintf("%#v", this.PostingsFetchedSizeSum)+",\n") + s = append(s, "PostingsFetchCount: "+fmt.Sprintf("%#v", this.PostingsFetchCount)+",\n") + s = append(s, "SeriesTouched: "+fmt.Sprintf("%#v", this.SeriesTouched)+",\n") + s = append(s, "SeriesTouchedSizeSum: "+fmt.Sprintf("%#v", this.SeriesTouchedSizeSum)+",\n") + s = append(s, "SeriesFetched: "+fmt.Sprintf("%#v", this.SeriesFetched)+",\n") + s = append(s, "SeriesFetchedSizeSum: "+fmt.Sprintf("%#v", this.SeriesFetchedSizeSum)+",\n") + s = append(s, "SeriesFetchCount: "+fmt.Sprintf("%#v", this.SeriesFetchCount)+",\n") + s = append(s, "ChunksTouched: "+fmt.Sprintf("%#v", this.ChunksTouched)+",\n") + s = append(s, "ChunksTouchedSizeSum: "+fmt.Sprintf("%#v", this.ChunksTouchedSizeSum)+",\n") + s = append(s, "ChunksFetched: "+fmt.Sprintf("%#v", this.ChunksFetched)+",\n") + s = append(s, "ChunksFetchedSizeSum: "+fmt.Sprintf("%#v", this.ChunksFetchedSizeSum)+",\n") + s = append(s, "ChunksFetchCount: "+fmt.Sprintf("%#v", this.ChunksFetchCount)+",\n") + s = append(s, "DataDownloadedSizeSum: "+fmt.Sprintf("%#v", this.DataDownloadedSizeSum)+",\n") + s = append(s, "GetAllDuration: "+fmt.Sprintf("%#v", this.GetAllDuration)+",\n") + s = append(s, "MergeDuration: "+fmt.Sprintf("%#v", this.MergeDuration)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringHints(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *SeriesRequestHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesRequestHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesRequestHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.EnableQueryStats { + i-- + if m.EnableQueryStats { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if len(m.BlockMatchers) > 0 { + for iNdEx := len(m.BlockMatchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BlockMatchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SeriesResponseHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesResponseHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesResponseHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.QueryStats != nil { + { + size, err := m.QueryStats.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.QueriedBlocks) > 0 { + for iNdEx := len(m.QueriedBlocks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.QueriedBlocks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Block) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Block) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Block) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = encodeVarintHints(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *LabelNamesRequestHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelNamesRequestHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelNamesRequestHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.BlockMatchers) > 0 { + for iNdEx := len(m.BlockMatchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BlockMatchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *LabelNamesResponseHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelNamesResponseHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelNamesResponseHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.QueriedBlocks) > 0 { + for iNdEx := len(m.QueriedBlocks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.QueriedBlocks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *LabelValuesRequestHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelValuesRequestHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelValuesRequestHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.BlockMatchers) > 0 { + for iNdEx := len(m.BlockMatchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BlockMatchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *LabelValuesResponseHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelValuesResponseHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelValuesResponseHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.QueriedBlocks) > 0 { + for iNdEx := len(m.QueriedBlocks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.QueriedBlocks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHints(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *QueryStats) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QueryStats) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryStats) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + n2, err2 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.MergeDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.MergeDuration):]) + if err2 != nil { + return 0, err2 + } + i -= n2 + i = encodeVarintHints(dAtA, i, uint64(n2)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xb2 + n3, err3 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.GetAllDuration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.GetAllDuration):]) + if err3 != nil { + return 0, err3 + } + i -= n3 + i = encodeVarintHints(dAtA, i, uint64(n3)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + if m.DataDownloadedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.DataDownloadedSizeSum)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa0 + } + if m.ChunksFetchCount != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.ChunksFetchCount)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x98 + } + if m.ChunksFetchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.ChunksFetchedSizeSum)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x90 + } + if m.ChunksFetched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.ChunksFetched)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x88 + } + if m.ChunksTouchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.ChunksTouchedSizeSum)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 + } + if m.ChunksTouched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.ChunksTouched)) + i-- + dAtA[i] = 0x78 + } + if m.SeriesFetchCount != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.SeriesFetchCount)) + i-- + dAtA[i] = 0x70 + } + if m.SeriesFetchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.SeriesFetchedSizeSum)) + i-- + dAtA[i] = 0x68 + } + if m.SeriesFetched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.SeriesFetched)) + i-- + dAtA[i] = 0x60 + } + if m.SeriesTouchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.SeriesTouchedSizeSum)) + i-- + dAtA[i] = 0x58 + } + if m.SeriesTouched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.SeriesTouched)) + i-- + dAtA[i] = 0x50 + } + if m.PostingsFetchCount != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsFetchCount)) + i-- + dAtA[i] = 0x48 + } + if m.PostingsFetchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsFetchedSizeSum)) + i-- + dAtA[i] = 0x40 + } + if m.PostingsFetched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsFetched)) + i-- + dAtA[i] = 0x38 + } + if m.PostingsToFetch != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsToFetch)) + i-- + dAtA[i] = 0x30 + } + if m.PostingsTouchedSizeSum != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsTouchedSizeSum)) + i-- + dAtA[i] = 0x28 + } + if m.PostingsTouched != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.PostingsTouched)) + i-- + dAtA[i] = 0x20 + } + if m.MergedChunksCount != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.MergedChunksCount)) + i-- + dAtA[i] = 0x18 + } + if m.MergedSeriesCount != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.MergedSeriesCount)) + i-- + dAtA[i] = 0x10 + } + if m.BlocksQueried != 0 { + i = encodeVarintHints(dAtA, i, uint64(m.BlocksQueried)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintHints(dAtA []byte, offset int, v uint64) int { + offset -= sovHints(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *SeriesRequestHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.BlockMatchers) > 0 { + for _, e := range m.BlockMatchers { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + if m.EnableQueryStats { + n += 2 + } + return n +} + +func (m *SeriesResponseHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.QueriedBlocks) > 0 { + for _, e := range m.QueriedBlocks { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + if m.QueryStats != nil { + l = m.QueryStats.Size() + n += 1 + l + sovHints(uint64(l)) + } + return n +} + +func (m *Block) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Id) + if l > 0 { + n += 1 + l + sovHints(uint64(l)) + } + return n +} + +func (m *LabelNamesRequestHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.BlockMatchers) > 0 { + for _, e := range m.BlockMatchers { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + return n +} + +func (m *LabelNamesResponseHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.QueriedBlocks) > 0 { + for _, e := range m.QueriedBlocks { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + return n +} + +func (m *LabelValuesRequestHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.BlockMatchers) > 0 { + for _, e := range m.BlockMatchers { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + return n +} + +func (m *LabelValuesResponseHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.QueriedBlocks) > 0 { + for _, e := range m.QueriedBlocks { + l = e.Size() + n += 1 + l + sovHints(uint64(l)) + } + } + return n +} + +func (m *QueryStats) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BlocksQueried != 0 { + n += 1 + sovHints(uint64(m.BlocksQueried)) + } + if m.MergedSeriesCount != 0 { + n += 1 + sovHints(uint64(m.MergedSeriesCount)) + } + if m.MergedChunksCount != 0 { + n += 1 + sovHints(uint64(m.MergedChunksCount)) + } + if m.PostingsTouched != 0 { + n += 1 + sovHints(uint64(m.PostingsTouched)) + } + if m.PostingsTouchedSizeSum != 0 { + n += 1 + sovHints(uint64(m.PostingsTouchedSizeSum)) + } + if m.PostingsToFetch != 0 { + n += 1 + sovHints(uint64(m.PostingsToFetch)) + } + if m.PostingsFetched != 0 { + n += 1 + sovHints(uint64(m.PostingsFetched)) + } + if m.PostingsFetchedSizeSum != 0 { + n += 1 + sovHints(uint64(m.PostingsFetchedSizeSum)) + } + if m.PostingsFetchCount != 0 { + n += 1 + sovHints(uint64(m.PostingsFetchCount)) + } + if m.SeriesTouched != 0 { + n += 1 + sovHints(uint64(m.SeriesTouched)) + } + if m.SeriesTouchedSizeSum != 0 { + n += 1 + sovHints(uint64(m.SeriesTouchedSizeSum)) + } + if m.SeriesFetched != 0 { + n += 1 + sovHints(uint64(m.SeriesFetched)) + } + if m.SeriesFetchedSizeSum != 0 { + n += 1 + sovHints(uint64(m.SeriesFetchedSizeSum)) + } + if m.SeriesFetchCount != 0 { + n += 1 + sovHints(uint64(m.SeriesFetchCount)) + } + if m.ChunksTouched != 0 { + n += 1 + sovHints(uint64(m.ChunksTouched)) + } + if m.ChunksTouchedSizeSum != 0 { + n += 2 + sovHints(uint64(m.ChunksTouchedSizeSum)) + } + if m.ChunksFetched != 0 { + n += 2 + sovHints(uint64(m.ChunksFetched)) + } + if m.ChunksFetchedSizeSum != 0 { + n += 2 + sovHints(uint64(m.ChunksFetchedSizeSum)) + } + if m.ChunksFetchCount != 0 { + n += 2 + sovHints(uint64(m.ChunksFetchCount)) + } + if m.DataDownloadedSizeSum != 0 { + n += 2 + sovHints(uint64(m.DataDownloadedSizeSum)) + } + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.GetAllDuration) + n += 2 + l + sovHints(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.MergeDuration) + n += 2 + l + sovHints(uint64(l)) + return n +} + +func sovHints(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozHints(x uint64) (n int) { + return sovHints(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *SeriesRequestHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForBlockMatchers := "[]LabelMatcher{" + for _, f := range this.BlockMatchers { + repeatedStringForBlockMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForBlockMatchers += "}" + s := strings.Join([]string{`&SeriesRequestHints{`, + `BlockMatchers:` + repeatedStringForBlockMatchers + `,`, + `EnableQueryStats:` + fmt.Sprintf("%v", this.EnableQueryStats) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponseHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForQueriedBlocks := "[]Block{" + for _, f := range this.QueriedBlocks { + repeatedStringForQueriedBlocks += strings.Replace(strings.Replace(f.String(), "Block", "Block", 1), `&`, ``, 1) + "," + } + repeatedStringForQueriedBlocks += "}" + s := strings.Join([]string{`&SeriesResponseHints{`, + `QueriedBlocks:` + repeatedStringForQueriedBlocks + `,`, + `QueryStats:` + strings.Replace(this.QueryStats.String(), "QueryStats", "QueryStats", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Block) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Block{`, + `Id:` + fmt.Sprintf("%v", this.Id) + `,`, + `}`, + }, "") + return s +} +func (this *LabelNamesRequestHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForBlockMatchers := "[]LabelMatcher{" + for _, f := range this.BlockMatchers { + repeatedStringForBlockMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForBlockMatchers += "}" + s := strings.Join([]string{`&LabelNamesRequestHints{`, + `BlockMatchers:` + repeatedStringForBlockMatchers + `,`, + `}`, + }, "") + return s +} +func (this *LabelNamesResponseHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForQueriedBlocks := "[]Block{" + for _, f := range this.QueriedBlocks { + repeatedStringForQueriedBlocks += strings.Replace(strings.Replace(f.String(), "Block", "Block", 1), `&`, ``, 1) + "," + } + repeatedStringForQueriedBlocks += "}" + s := strings.Join([]string{`&LabelNamesResponseHints{`, + `QueriedBlocks:` + repeatedStringForQueriedBlocks + `,`, + `}`, + }, "") + return s +} +func (this *LabelValuesRequestHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForBlockMatchers := "[]LabelMatcher{" + for _, f := range this.BlockMatchers { + repeatedStringForBlockMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForBlockMatchers += "}" + s := strings.Join([]string{`&LabelValuesRequestHints{`, + `BlockMatchers:` + repeatedStringForBlockMatchers + `,`, + `}`, + }, "") + return s +} +func (this *LabelValuesResponseHints) String() string { + if this == nil { + return "nil" + } + repeatedStringForQueriedBlocks := "[]Block{" + for _, f := range this.QueriedBlocks { + repeatedStringForQueriedBlocks += strings.Replace(strings.Replace(f.String(), "Block", "Block", 1), `&`, ``, 1) + "," + } + repeatedStringForQueriedBlocks += "}" + s := strings.Join([]string{`&LabelValuesResponseHints{`, + `QueriedBlocks:` + repeatedStringForQueriedBlocks + `,`, + `}`, + }, "") + return s +} +func (this *QueryStats) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryStats{`, + `BlocksQueried:` + fmt.Sprintf("%v", this.BlocksQueried) + `,`, + `MergedSeriesCount:` + fmt.Sprintf("%v", this.MergedSeriesCount) + `,`, + `MergedChunksCount:` + fmt.Sprintf("%v", this.MergedChunksCount) + `,`, + `PostingsTouched:` + fmt.Sprintf("%v", this.PostingsTouched) + `,`, + `PostingsTouchedSizeSum:` + fmt.Sprintf("%v", this.PostingsTouchedSizeSum) + `,`, + `PostingsToFetch:` + fmt.Sprintf("%v", this.PostingsToFetch) + `,`, + `PostingsFetched:` + fmt.Sprintf("%v", this.PostingsFetched) + `,`, + `PostingsFetchedSizeSum:` + fmt.Sprintf("%v", this.PostingsFetchedSizeSum) + `,`, + `PostingsFetchCount:` + fmt.Sprintf("%v", this.PostingsFetchCount) + `,`, + `SeriesTouched:` + fmt.Sprintf("%v", this.SeriesTouched) + `,`, + `SeriesTouchedSizeSum:` + fmt.Sprintf("%v", this.SeriesTouchedSizeSum) + `,`, + `SeriesFetched:` + fmt.Sprintf("%v", this.SeriesFetched) + `,`, + `SeriesFetchedSizeSum:` + fmt.Sprintf("%v", this.SeriesFetchedSizeSum) + `,`, + `SeriesFetchCount:` + fmt.Sprintf("%v", this.SeriesFetchCount) + `,`, + `ChunksTouched:` + fmt.Sprintf("%v", this.ChunksTouched) + `,`, + `ChunksTouchedSizeSum:` + fmt.Sprintf("%v", this.ChunksTouchedSizeSum) + `,`, + `ChunksFetched:` + fmt.Sprintf("%v", this.ChunksFetched) + `,`, + `ChunksFetchedSizeSum:` + fmt.Sprintf("%v", this.ChunksFetchedSizeSum) + `,`, + `ChunksFetchCount:` + fmt.Sprintf("%v", this.ChunksFetchCount) + `,`, + `DataDownloadedSizeSum:` + fmt.Sprintf("%v", this.DataDownloadedSizeSum) + `,`, + `GetAllDuration:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.GetAllDuration), "Duration", "duration.Duration", 1), `&`, ``, 1) + `,`, + `MergeDuration:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.MergeDuration), "Duration", "duration.Duration", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringHints(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *SeriesRequestHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesRequestHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesRequestHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockMatchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockMatchers = append(m.BlockMatchers, typespb.LabelMatcher{}) + if err := m.BlockMatchers[len(m.BlockMatchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EnableQueryStats", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EnableQueryStats = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SeriesResponseHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesResponseHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesResponseHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueriedBlocks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.QueriedBlocks = append(m.QueriedBlocks, Block{}) + if err := m.QueriedBlocks[len(m.QueriedBlocks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryStats", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.QueryStats == nil { + m.QueryStats = &QueryStats{} + } + if err := m.QueryStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Block) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Block: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Block: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelNamesRequestHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelNamesRequestHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelNamesRequestHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockMatchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockMatchers = append(m.BlockMatchers, typespb.LabelMatcher{}) + if err := m.BlockMatchers[len(m.BlockMatchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelNamesResponseHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelNamesResponseHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelNamesResponseHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueriedBlocks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.QueriedBlocks = append(m.QueriedBlocks, Block{}) + if err := m.QueriedBlocks[len(m.QueriedBlocks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelValuesRequestHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelValuesRequestHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelValuesRequestHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockMatchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockMatchers = append(m.BlockMatchers, typespb.LabelMatcher{}) + if err := m.BlockMatchers[len(m.BlockMatchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelValuesResponseHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelValuesResponseHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelValuesResponseHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueriedBlocks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.QueriedBlocks = append(m.QueriedBlocks, Block{}) + if err := m.QueriedBlocks[len(m.QueriedBlocks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *QueryStats) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: QueryStats: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QueryStats: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlocksQueried", wireType) + } + m.BlocksQueried = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlocksQueried |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MergedSeriesCount", wireType) + } + m.MergedSeriesCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MergedSeriesCount |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MergedChunksCount", wireType) + } + m.MergedChunksCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MergedChunksCount |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsTouched", wireType) + } + m.PostingsTouched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsTouched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsTouchedSizeSum", wireType) + } + m.PostingsTouchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsTouchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsToFetch", wireType) + } + m.PostingsToFetch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsToFetch |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsFetched", wireType) + } + m.PostingsFetched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsFetched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsFetchedSizeSum", wireType) + } + m.PostingsFetchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsFetchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostingsFetchCount", wireType) + } + m.PostingsFetchCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostingsFetchCount |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesTouched", wireType) + } + m.SeriesTouched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesTouched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesTouchedSizeSum", wireType) + } + m.SeriesTouchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesTouchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesFetched", wireType) + } + m.SeriesFetched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesFetched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesFetchedSizeSum", wireType) + } + m.SeriesFetchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesFetchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesFetchCount", wireType) + } + m.SeriesFetchCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesFetchCount |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunksTouched", wireType) + } + m.ChunksTouched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunksTouched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 16: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunksTouchedSizeSum", wireType) + } + m.ChunksTouchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunksTouchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 17: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunksFetched", wireType) + } + m.ChunksFetched = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunksFetched |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 18: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunksFetchedSizeSum", wireType) + } + m.ChunksFetchedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunksFetchedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 19: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunksFetchCount", wireType) + } + m.ChunksFetchCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunksFetchCount |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 20: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DataDownloadedSizeSum", wireType) + } + m.DataDownloadedSizeSum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DataDownloadedSizeSum |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 21: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GetAllDuration", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.GetAllDuration, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 22: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MergeDuration", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHints + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHints + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.MergeDuration, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipHints(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowHints + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowHints + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowHints + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthHints + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthHints + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowHints + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipHints(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthHints + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthHints = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowHints = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/storegateway/hintspb/hints.proto b/pkg/storegateway/hintspb/hints.proto new file mode 100644 index 0000000000..7dffc474f7 --- /dev/null +++ b/pkg/storegateway/hintspb/hints.proto @@ -0,0 +1,96 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +syntax = "proto3"; +package hintspb; + +import "gogoproto/gogo.proto"; +import "google/protobuf/duration.proto"; +import "github.com/cortexproject/cortex/pkg/storegateway/typespb/types.proto"; + +option go_package = "hintspb"; + +option (gogoproto.sizer_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +// Do not generate XXX fields to reduce memory. +option (gogoproto.goproto_unkeyed_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.goproto_sizecache_all) = false; + +message SeriesRequestHints { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + repeated typespb.LabelMatcher block_matchers = 1 [(gogoproto.nullable) = false]; + + bool enable_query_stats = 2; +} + +message SeriesResponseHints { + /// queried_blocks is the list of blocks that have been queried. + repeated Block queried_blocks = 1 [(gogoproto.nullable) = false]; + /// query_stats contains statistics of querying store gateway. + QueryStats query_stats = 2 [(gogoproto.nullable) = true]; +} + +message Block { + string id = 1; +} + + +message LabelNamesRequestHints { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + repeated typespb.LabelMatcher block_matchers = 1 [(gogoproto.nullable) = false]; +} + +message LabelNamesResponseHints { + /// queried_blocks is the list of blocks that have been queried. + repeated Block queried_blocks = 1 [(gogoproto.nullable) = false]; +} + +message LabelValuesRequestHints { + /// block_matchers is a list of label matchers that are evaluated against each single block's + /// labels to filter which blocks get queried. If the list is empty, no per-block filtering + /// is applied. + repeated typespb.LabelMatcher block_matchers = 1 [(gogoproto.nullable) = false]; +} + +message LabelValuesResponseHints { + /// queried_blocks is the list of blocks that have been queried. + repeated Block queried_blocks = 1 [(gogoproto.nullable) = false]; +} + +/// QueryStats fields are unstable and might change in the future. +message QueryStats { + int64 blocks_queried = 1; + int64 merged_series_count = 2; + int64 merged_chunks_count = 3; + + int64 postings_touched = 4; + int64 postings_touched_size_sum = 5; + int64 postings_to_fetch = 6; + int64 postings_fetched = 7; + int64 postings_fetched_size_sum = 8; + int64 postings_fetch_count = 9; + + int64 series_touched = 10; + int64 series_touched_size_sum = 11; + int64 series_fetched = 12; + int64 series_fetched_size_sum = 13; + int64 series_fetch_count = 14; + + int64 chunks_touched = 15; + int64 chunks_touched_size_sum = 16; + int64 chunks_fetched = 17; + int64 chunks_fetched_size_sum = 18; + int64 chunks_fetch_count = 19; + + int64 data_downloaded_size_sum = 20; + google.protobuf.Duration get_all_duration = 21 [(gogoproto.stdduration) = true, (gogoproto.nullable) = false]; + google.protobuf.Duration merge_duration = 22 [(gogoproto.stdduration) = true, (gogoproto.nullable) = false]; +} \ No newline at end of file diff --git a/pkg/storegateway/io.go b/pkg/storegateway/io.go new file mode 100644 index 0000000000..28be720be3 --- /dev/null +++ b/pkg/storegateway/io.go @@ -0,0 +1,125 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "bufio" + "io" + + "github.com/pkg/errors" +) + +const ( + readerBufferSize = 32 * 1024 +) + +// byteRange holds information about a single byte range. +type byteRange struct { + offset int + length int +} + +// byteRanges holds a list of non-overlapping byte ranges sorted by offset. +type byteRanges []byteRange + +// size returns the total number of bytes in the byte ranges. +func (r byteRanges) size() int { + size := 0 + for _, c := range r { + size += c.length + } + return size +} + +// areContiguous returns whether all byte ranges are contiguous (no gaps). +func (r byteRanges) areContiguous() bool { + if len(r) < 2 { + return true + } + + for off, idx := r[0].offset+r[0].length, 1; idx < len(r); idx++ { + if r[idx].offset != off { + return false + } + off += r[idx].length + } + return true +} + +// readByteRanges reads the provided byteRanges from src and append them to dst. The provided +// byteRanges must be sorted by offset and non overlapping. The byteRanges offset must be +// relative to the beginning of the provided src (offset 0 == first byte will be read from src). +func readByteRanges(src io.Reader, dst []byte, byteRanges byteRanges) ([]byte, error) { + if len(byteRanges) == 0 { + return nil, nil + } + + // Ensure the provided dst buffer has enough capacity. + expectedSize := byteRanges.size() + if cap(dst) < expectedSize { + return nil, io.ErrShortBuffer + } + + // Size the destination buffer accordingly. + dst = dst[0:expectedSize] + + // Optimisation for the case all ranges are contiguous. + if byteRanges[0].offset == 0 && byteRanges.areContiguous() { + // We get an ErrUnexpectedEOF if EOF is reached before we fill allocated dst slice. + // Due to how the reading logic works in the bucket store, we may try to overread at + // the end of an object, so we consider it legit. + if _, err := io.ReadFull(src, dst); err != nil && err != io.ErrUnexpectedEOF { + return nil, err + } + return dst, nil + } + + // To keep implementation easier we frequently call Read() for short lengths. + // In such scenario, having a buffered reader improves performances at the cost + // of 1 more buffer allocation and memory copy. + reader := bufio.NewReaderSize(src, readerBufferSize) + + for dstOffset, idx := 0, 0; idx < len(byteRanges); idx++ { + curr := byteRanges[idx] + + // Read and discard all bytes before the current chunk offset. + discard := 0 + if idx == 0 { + discard = curr.offset + } else { + prev := byteRanges[idx-1] + discard = curr.offset - (prev.offset + prev.length) + } + + if _, err := reader.Discard(discard); err != nil { + if err == io.EOF { + err = io.ErrUnexpectedEOF + } + return nil, errors.Wrap(err, "discard bytes") + } + + // At this point the next byte to read from the reader is the current chunk, + // so we'll read it fully. io.ReadFull() returns an error if less bytes than + // expected have been read. + readBytes, err := io.ReadFull(reader, dst[dstOffset:dstOffset+curr.length]) + if readBytes > 0 { + dstOffset += readBytes + } + if err != nil { + // We get an ErrUnexpectedEOF if EOF is reached before we fill the slice. + // Due to how the reading logic works in the bucket store, we may try to overread + // the last byte range so, if the error occurrs on the last one, we consider it legit. + if err == io.ErrUnexpectedEOF && idx == len(byteRanges)-1 { + return dst, nil + } + + if err == io.EOF { + err = io.ErrUnexpectedEOF + } + return nil, errors.Wrap(err, "read byte range") + } + } + + return dst, nil +} diff --git a/pkg/storegateway/lazy_postings.go b/pkg/storegateway/lazy_postings.go new file mode 100644 index 0000000000..13f8d77c71 --- /dev/null +++ b/pkg/storegateway/lazy_postings.go @@ -0,0 +1,271 @@ +package storegateway + +import ( + "context" + "math" + "strings" + + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/index" + "golang.org/x/exp/slices" + + "github.com/thanos-io/thanos/pkg/block/indexheader" +) + +var emptyLazyPostings = &lazyExpandedPostings{postings: nil, matchers: nil} + +// lazyExpandedPostings contains expanded postings (series IDs). If lazy posting expansion is +// enabled, it might contain matchers that can be lazily applied during series filtering time. +type lazyExpandedPostings struct { + postings []storage.SeriesRef + matchers []*labels.Matcher +} + +func newLazyExpandedPostings(ps []storage.SeriesRef, matchers ...*labels.Matcher) *lazyExpandedPostings { + return &lazyExpandedPostings{ + postings: ps, + matchers: matchers, + } +} + +func (p *lazyExpandedPostings) lazyExpanded() bool { + return p != nil && len(p.matchers) > 0 +} + +func optimizePostingsFetchByDownloadedBytes(r *bucketIndexReader, postingGroups []*postingGroup, seriesMaxSize int64, seriesMatchRatio float64, lazyExpandedPostingSizeBytes prometheus.Counter) ([]*postingGroup, bool, error) { + if len(postingGroups) <= 1 { + return postingGroups, false, nil + } + // Collect posting cardinality of each posting group. + for _, pg := range postingGroups { + // A posting group can have either add keys or remove keys but not both the same time. + vals := pg.addKeys + if len(pg.removeKeys) > 0 { + vals = pg.removeKeys + } + rngs, err := r.block.indexHeaderReader.PostingsOffsets(pg.name, vals...) + if err != nil { + return nil, false, errors.Wrapf(err, "postings offsets for %s", pg.name) + } + + // No posting ranges found means empty posting. + if len(rngs) == 0 { + return nil, true, nil + } + for _, r := range rngs { + if r == indexheader.NotFoundRange { + continue + } + // Each range starts from the #entries field which is 4 bytes. + // Need to subtract it when calculating number of postings. + // https://github.com/prometheus/prometheus/blob/v2.46.0/tsdb/docs/format/index.md. + pg.cardinality += (r.End - r.Start - 4) / 4 + } + } + slices.SortFunc(postingGroups, func(a, b *postingGroup) int { + if a.cardinality == b.cardinality { + return strings.Compare(a.name, b.name) + } + return int(a.cardinality - b.cardinality) + }) + + /* + Algorithm of choosing what postings we need to fetch right now and what + postings we expand lazily. + Sort posting groups by cardinality, so we can iterate from posting group with the smallest posting size. + The algorithm focuses on fetching fewer data, including postings and series. + + We need to fetch at least 1 posting group in order to fetch series. So if we only fetch the first posting group, + the data bytes we need to download is formula F1: P1 * 4 + P1 * S where P1 is the number of postings in group 1 + and S is the size per series. 4 is the byte size per posting. + + If we are going to fetch 2 posting groups, we can intersect the two postings to reduce series we need to download (hopefully). + Assuming for each intersection, the series matching ratio is R (0 < R < 1). Then the data bytes we need to download is + formula F2: P1 * 4 + P2 * 4 + P1 * S * R. + We can get formula F3 if we are going to fetch 3 posting groups: + F3: P1 * 4 + P2 * 4 + P3 * 4 + P1 * S * R^2. + + Let's compare formula F2 and F1 first. + P1 * 4 + P2 * 4 + P1 * S * R < P1 * 4 + P1 * S + => P2 * 4 < P1 * S * (1 - R) + Left hand side is the posting group size and right hand side is basically the series size we don't need to fetch + by having the additional intersection. In order to fetch less data for F2 than F1, we just need to ensure that + the additional postings size is smaller. + + Let's compare formula F3 and F2. + P1 * 4 + P2 * 4 + P3 * 4 + P1 * S * R^2 < P1 * 4 + P2 * 4 + P1 * S * R + => P3 * 4 < P1 * S * R * (1 - R) + Same as the previous formula. + + Compare formula F4 (Cost to fetch up to 4 posting groups) and F3. + P4 * 4 < P1 * S * R^2 * (1 - R) + + We can generalize this to formula: Pn * 4 < P1 * S * R^(n - 2) * (1 - R) + + The idea of the algorithm: + By iterating the posting group in sorted order of cardinality, we need to make sure that by fetching the current posting group, + the total data fetched is smaller than the previous posting group. If so, then we continue to next posting group, + otherwise we stop. + + This ensures that when we stop at one posting group, posting groups after it always need to fetch more data. + Based on formula Pn * 4 < P1 * S * R^(n - 2) * (1 - R), left hand side is always increasing while iterating to larger + posting groups while right hand side value is always decreasing as R < 1. + */ + seriesBytesToFetch := postingGroups[0].cardinality * seriesMaxSize + p := float64(1) + i := 1 // Start from index 1 as we always need to fetch the smallest posting group. + hasAdd := !postingGroups[0].addAll + for i < len(postingGroups) { + pg := postingGroups[i] + // Need to fetch more data on postings than series we avoid fetching, stop here and lazy expanding rest of matchers. + // If there is no posting group with add keys, don't skip any posting group until we have one. + // Fetch posting group with addAll is much more expensive due to fetch all postings. + if hasAdd && pg.cardinality*4 > int64(p*math.Ceil((1-seriesMatchRatio)*float64(seriesBytesToFetch))) { + break + } + hasAdd = hasAdd || !pg.addAll + p = p * seriesMatchRatio + i++ + } + for i < len(postingGroups) { + postingGroups[i].lazy = true + lazyExpandedPostingSizeBytes.Add(float64(4 * postingGroups[i].cardinality)) + i++ + } + return postingGroups, false, nil +} + +func fetchLazyExpandedPostings( + ctx context.Context, + postingGroups []*postingGroup, + r *bucketIndexReader, + bytesLimiter BytesLimiter, + addAllPostings bool, + lazyExpandedPostingEnabled bool, + lazyExpandedPostingSizeBytes prometheus.Counter, + tenant string, +) (*lazyExpandedPostings, error) { + var ( + err error + emptyPostingGroup bool + ) + /* + There are several cases that we skip postings fetch optimization: + - Lazy expanded posting disabled. + - Add all postings. This means we don't have a posting group with any add keys. + - Block estimated max series size not set which means we don't have a way to estimate series bytes downloaded. + - `SeriesMaxSize` not set for this block then we have no way to estimate series size. + - Only one effective posting group available. We need to at least download postings from 1 posting group so no need to optimize. + */ + if lazyExpandedPostingEnabled && !addAllPostings && + r.block.estimatedMaxSeriesSize > 0 && len(postingGroups) > 1 { + postingGroups, emptyPostingGroup, err = optimizePostingsFetchByDownloadedBytes( + r, + postingGroups, + int64(r.block.estimatedMaxSeriesSize), + 0.5, // TODO(yeya24): Expose this as a flag. + lazyExpandedPostingSizeBytes, + ) + if err != nil { + return nil, err + } + if emptyPostingGroup { + return emptyLazyPostings, nil + } + } + + ps, matchers, err := fetchAndExpandPostingGroups(ctx, r, postingGroups, bytesLimiter, tenant) + if err != nil { + return nil, err + } + return &lazyExpandedPostings{postings: ps, matchers: matchers}, nil +} + +// keysToFetchFromPostingGroups returns label pairs (postings) to fetch +// and matchers we need to use for lazy posting expansion. +// Input `postingGroups` needs to be ordered by cardinality in case lazy +// expansion is enabled. When we find the first lazy posting group we can exit. +func keysToFetchFromPostingGroups(postingGroups []*postingGroup) ([]labels.Label, []*labels.Matcher) { + var lazyMatchers []*labels.Matcher + keys := make([]labels.Label, 0) + i := 0 + for i < len(postingGroups) { + pg := postingGroups[i] + if pg.lazy { + break + } + + // Postings returned by fetchPostings will be in the same order as keys + // so it's important that we iterate them in the same order later. + // We don't have any other way of pairing keys and fetched postings. + for _, key := range pg.addKeys { + keys = append(keys, labels.Label{Name: pg.name, Value: key}) + } + for _, key := range pg.removeKeys { + keys = append(keys, labels.Label{Name: pg.name, Value: key}) + } + i++ + } + if i < len(postingGroups) { + lazyMatchers = make([]*labels.Matcher, 0) + for i < len(postingGroups) { + lazyMatchers = append(lazyMatchers, postingGroups[i].matchers...) + i++ + } + } + return keys, lazyMatchers +} + +func fetchAndExpandPostingGroups(ctx context.Context, r *bucketIndexReader, postingGroups []*postingGroup, bytesLimiter BytesLimiter, tenant string) ([]storage.SeriesRef, []*labels.Matcher, error) { + keys, lazyMatchers := keysToFetchFromPostingGroups(postingGroups) + fetchedPostings, closeFns, err := r.fetchPostings(ctx, keys, bytesLimiter, tenant) + defer func() { + for _, closeFn := range closeFns { + closeFn() + } + }() + if err != nil { + return nil, nil, errors.Wrap(err, "get postings") + } + + // Get "add" and "remove" postings from groups. We iterate over postingGroups and their keys + // again, and this is exactly the same order as before (when building the groups), so we can simply + // use one incrementing index to fetch postings from returned slice. + postingIndex := 0 + + var groupAdds, groupRemovals []index.Postings + for _, g := range postingGroups { + if g.lazy { + break + } + // We cannot add empty set to groupAdds, since they are intersected. + if len(g.addKeys) > 0 { + toMerge := make([]index.Postings, 0, len(g.addKeys)) + for _, l := range g.addKeys { + toMerge = append(toMerge, checkNilPosting(g.name, l, fetchedPostings[postingIndex])) + postingIndex++ + } + + groupAdds = append(groupAdds, index.Merge(ctx, toMerge...)) + } + + for _, l := range g.removeKeys { + groupRemovals = append(groupRemovals, checkNilPosting(g.name, l, fetchedPostings[postingIndex])) + postingIndex++ + } + } + + result := index.Without(index.Intersect(groupAdds...), index.Merge(ctx, groupRemovals...)) + + if ctx.Err() != nil { + return nil, nil, ctx.Err() + } + ps, err := ExpandPostingsWithContext(ctx, result) + if err != nil { + return nil, nil, errors.Wrap(err, "expand") + } + return ps, lazyMatchers, nil +} diff --git a/pkg/storegateway/limiter.go b/pkg/storegateway/limiter.go new file mode 100644 index 0000000000..b697d46057 --- /dev/null +++ b/pkg/storegateway/limiter.go @@ -0,0 +1,176 @@ +package storegateway + +import ( + "sync" + + "github.com/alecthomas/units" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "go.uber.org/atomic" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/thanos-io/thanos/pkg/extkingpin" +) + +type ChunksLimiter interface { + // Reserve num chunks out of the total number of chunks enforced by the limiter. + // Returns an error if the limit has been exceeded. This function must be + // goroutine safe. + Reserve(num uint64) error +} + +type SeriesLimiter interface { + // Reserve num series out of the total number of series enforced by the limiter. + // Returns an error if the limit has been exceeded. This function must be + // goroutine safe. + Reserve(num uint64) error +} + +type BytesLimiter interface { + // Reserve bytes out of the total amount of bytes enforced by the limiter. + // Returns an error if the limit has been exceeded. This function must be + // goroutine safe. + Reserve(num uint64) error +} + +// ChunksLimiterFactory is used to create a new ChunksLimiter. The factory is useful for +// projects depending on Thanos (eg. Cortex) which have dynamic limits. +type ChunksLimiterFactory func(failedCounter prometheus.Counter) ChunksLimiter + +// SeriesLimiterFactory is used to create a new SeriesLimiter. +type SeriesLimiterFactory func(failedCounter prometheus.Counter) SeriesLimiter + +// BytesLimiterFactory is used to create a new BytesLimiter. +type BytesLimiterFactory func(failedCounter prometheus.Counter) BytesLimiter + +// Limiter is a simple mechanism for checking if something has passed a certain threshold. +type Limiter struct { + limit uint64 + reserved atomic.Uint64 + + // Counter metric which we will increase if limit is exceeded. + failedCounter prometheus.Counter + failedOnce sync.Once +} + +// NewLimiter returns a new limiter with a specified limit. 0 disables the limit. +func NewLimiter(limit uint64, ctr prometheus.Counter) *Limiter { + return &Limiter{limit: limit, failedCounter: ctr} +} + +// Reserve implements ChunksLimiter. +func (l *Limiter) Reserve(num uint64) error { + if l == nil { + return nil + } + if l.limit == 0 { + return nil + } + if reserved := l.reserved.Add(num); reserved > l.limit { + // We need to protect from the counter being incremented twice due to concurrency + // while calling Reserve(). + l.failedOnce.Do(l.failedCounter.Inc) + return errors.Errorf("limit %v violated (got %v)", l.limit, reserved) + } + return nil +} + +// NewChunksLimiterFactory makes a new ChunksLimiterFactory with a static limit. +func NewChunksLimiterFactory(limit uint64) ChunksLimiterFactory { + return func(failedCounter prometheus.Counter) ChunksLimiter { + return NewLimiter(limit, failedCounter) + } +} + +// NewSeriesLimiterFactory makes a new SeriesLimiterFactory with a static limit. +func NewSeriesLimiterFactory(limit uint64) SeriesLimiterFactory { + return func(failedCounter prometheus.Counter) SeriesLimiter { + return NewLimiter(limit, failedCounter) + } +} + +// NewBytesLimiterFactory makes a new BytesLimiterFactory with a static limit. +func NewBytesLimiterFactory(limit units.Base2Bytes) BytesLimiterFactory { + return func(failedCounter prometheus.Counter) BytesLimiter { + return NewLimiter(uint64(limit), failedCounter) + } +} + +// SeriesSelectLimits are limits applied against individual Series calls. +type SeriesSelectLimits struct { + SeriesPerRequest uint64 + SamplesPerRequest uint64 +} + +func (l *SeriesSelectLimits) RegisterFlags(cmd extkingpin.FlagClause) { + cmd.Flag("store.limits.request-series", "The maximum series allowed for a single Series request. The Series call fails if this limit is exceeded. 0 means no limit.").Default("0").Uint64Var(&l.SeriesPerRequest) + cmd.Flag("store.limits.request-samples", "The maximum samples allowed for a single Series request, The Series call fails if this limit is exceeded. 0 means no limit. NOTE: For efficiency the limit is internally implemented as 'chunks limit' considering each chunk contains a maximum of 120 samples.").Default("0").Uint64Var(&l.SamplesPerRequest) +} + +var _ storepb.StoreServer = &limitedStoreServer{} + +// limitedStoreServer is a storepb.StoreServer that can apply series and sample limits against individual Series requests. +type limitedStoreServer struct { + storepb.StoreServer + newSeriesLimiter SeriesLimiterFactory + newSamplesLimiter ChunksLimiterFactory + failedRequestsCounter *prometheus.CounterVec +} + +// NewLimitedStoreServer creates a new limitedStoreServer. +func NewLimitedStoreServer(store storepb.StoreServer, reg prometheus.Registerer, selectLimits SeriesSelectLimits) storepb.StoreServer { + return &limitedStoreServer{ + StoreServer: store, + newSeriesLimiter: NewSeriesLimiterFactory(selectLimits.SeriesPerRequest), + newSamplesLimiter: NewChunksLimiterFactory(selectLimits.SamplesPerRequest), + failedRequestsCounter: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "thanos_store_selects_dropped_total", + Help: "Number of select queries that were dropped due to configured limits.", + }, []string{"reason"}), + } +} + +func (s *limitedStoreServer) Series(req *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { + seriesLimiter := s.newSeriesLimiter(s.failedRequestsCounter.WithLabelValues("series")) + chunksLimiter := s.newSamplesLimiter(s.failedRequestsCounter.WithLabelValues("chunks")) + limitedSrv := newLimitedServer(srv, seriesLimiter, chunksLimiter) + if err := s.StoreServer.Series(req, limitedSrv); err != nil { + return err + } + + return nil +} + +var _ storepb.Store_SeriesServer = &limitedServer{} + +// limitedServer is a storepb.Store_SeriesServer that tracks statistics about sent series. +type limitedServer struct { + storepb.Store_SeriesServer + seriesLimiter SeriesLimiter + samplesLimiter ChunksLimiter +} + +func newLimitedServer(upstream storepb.Store_SeriesServer, seriesLimiter SeriesLimiter, chunksLimiter ChunksLimiter) *limitedServer { + return &limitedServer{ + Store_SeriesServer: upstream, + seriesLimiter: seriesLimiter, + samplesLimiter: chunksLimiter, + } +} + +func (i *limitedServer) Send(response *storepb.SeriesResponse) error { + series := response.GetSeries() + if series == nil { + return i.Store_SeriesServer.Send(response) + } + + if err := i.seriesLimiter.Reserve(1); err != nil { + return errors.Wrapf(err, "failed to send series") + } + if err := i.samplesLimiter.Reserve(uint64(len(series.Chunks) * MaxSamplesPerChunk)); err != nil { + return errors.Wrapf(err, "failed to send samples") + } + + return i.Store_SeriesServer.Send(response) +} diff --git a/pkg/storegateway/opts.go b/pkg/storegateway/opts.go new file mode 100644 index 0000000000..15b1f30196 --- /dev/null +++ b/pkg/storegateway/opts.go @@ -0,0 +1,73 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "strings" + "unicode/utf8" +) + +// Bitmap used by func isRegexMetaCharacter to check whether a character needs to be escaped. +var regexMetaCharacterBytes [16]byte + +// isRegexMetaCharacter reports whether byte b needs to be escaped. +func isRegexMetaCharacter(b byte) bool { + return b < utf8.RuneSelf && regexMetaCharacterBytes[b%16]&(1<<(b/16)) != 0 +} + +func init() { + for _, b := range []byte(`.+*?()|[]{}^$`) { + regexMetaCharacterBytes[b%16] |= 1 << (b / 16) + } +} + +// Copied from Prometheus querier.go, removed check for Prometheus wrapper. +// Returns list of values that can regex matches. +func findSetMatches(pattern string) []string { + if len(pattern) == 0 { + return nil + } + escaped := false + sets := []*strings.Builder{{}} + init := 0 + end := len(pattern) + // If the regex is wrapped in a group we can remove the first and last parentheses + if pattern[init] == '(' && pattern[end-1] == ')' { + init++ + end-- + } + for i := init; i < end; i++ { + if escaped { + switch { + case isRegexMetaCharacter(pattern[i]): + sets[len(sets)-1].WriteByte(pattern[i]) + case pattern[i] == '\\': + sets[len(sets)-1].WriteByte('\\') + default: + return nil + } + escaped = false + } else { + switch { + case isRegexMetaCharacter(pattern[i]): + if pattern[i] == '|' { + sets = append(sets, &strings.Builder{}) + } else { + return nil + } + case pattern[i] == '\\': + escaped = true + default: + sets[len(sets)-1].WriteByte(pattern[i]) + } + } + } + matches := make([]string, 0, len(sets)) + for _, s := range sets { + if s.Len() > 0 { + matches = append(matches, s.String()) + } + } + return matches +} diff --git a/pkg/storegateway/partitioner.go b/pkg/storegateway/partitioner.go index 816a45d8a5..07927de6c1 100644 --- a/pkg/storegateway/partitioner.go +++ b/pkg/storegateway/partitioner.go @@ -3,11 +3,25 @@ package storegateway import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/thanos-io/thanos/pkg/store" ) +type Part struct { + Start uint64 + End uint64 + + ElemRng [2]int +} + +type Partitioner interface { + // Partition partitions length entries into n <= length ranges that cover all + // input ranges + // It supports overlapping ranges. + // NOTE: It expects range to be sorted by start time. + Partition(length int, rng func(int) (uint64, uint64)) []Part +} + type gapBasedPartitioner struct { - upstream store.Partitioner + maxGapSize uint64 // Metrics. requestedBytes prometheus.Counter @@ -16,9 +30,9 @@ type gapBasedPartitioner struct { expandedRanges prometheus.Counter } -func newGapBasedPartitioner(maxGapBytes uint64, reg prometheus.Registerer) *gapBasedPartitioner { +func newGapBasedPartitioner(maxGapBytes uint64, reg prometheus.Registerer) Partitioner { return &gapBasedPartitioner{ - upstream: store.NewGapBasedPartitioner(maxGapBytes), + maxGapSize: maxGapBytes, requestedBytes: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "cortex_bucket_store_partitioner_requested_bytes_total", Help: "Total size of byte ranges required to fetch from the storage before they are passed to the partitioner.", @@ -38,7 +52,38 @@ func newGapBasedPartitioner(maxGapBytes uint64, reg prometheus.Registerer) *gapB } } -func (p *gapBasedPartitioner) Partition(length int, rng func(int) (uint64, uint64)) []store.Part { +// Partition partitions length entries into n <= length ranges that cover all +// input ranges by combining entries that are separated by reasonably small gaps. +// It is used to combine multiple small ranges from object storage into bigger, more efficient/cheaper ones. +func (g gapBasedPartitioner) partitionInternal(length int, rng func(int) (uint64, uint64)) (parts []Part) { + j := 0 + k := 0 + for k < length { + j = k + k++ + + p := Part{} + p.Start, p.End = rng(j) + + // Keep growing the range until the end or we encounter a large gap. + for ; k < length; k++ { + s, e := rng(k) + + if p.End+g.maxGapSize < s { + break + } + + if p.End <= e { + p.End = e + } + } + p.ElemRng = [2]int{j, k} + parts = append(parts, p) + } + return parts +} + +func (p *gapBasedPartitioner) Partition(length int, rng func(int) (uint64, uint64)) []Part { // Calculate the size of requested ranges. requestedBytes := uint64(0) for i := 0; i < length; i++ { @@ -47,7 +92,7 @@ func (p *gapBasedPartitioner) Partition(length int, rng func(int) (uint64, uint6 } // Run the upstream partitioner to compute the actual ranges that will be fetched. - parts := p.upstream.Partition(length, rng) + parts := p.partitionInternal(length, rng) // Calculate the size of ranges that will be fetched. expandedBytes := uint64(0) diff --git a/pkg/storegateway/postings.go b/pkg/storegateway/postings.go new file mode 100644 index 0000000000..203e1d2b63 --- /dev/null +++ b/pkg/storegateway/postings.go @@ -0,0 +1,143 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "bufio" + "context" + "encoding/binary" + "fmt" + "io" + + "github.com/pkg/errors" +) + +type postingsReaderBuilder struct { + e error + readBuf []byte + + r *bufio.Reader + postings []postingPtr + + lastOffset int64 + pi int + + start, length int64 + cur []byte + keyID int + repeatFor int + numberOfPostingsInCur uint64 + uvarintEncodeBuf []byte + ctx context.Context +} + +// newPostingsReaderBuilder is a builder that reads directly from the index +// and builds a diff varint encoded []byte that could be later used directly. +func newPostingsReaderBuilder(ctx context.Context, r *bufio.Reader, postings []postingPtr, start, length int64) *postingsReaderBuilder { + prb := &postingsReaderBuilder{ + r: r, + readBuf: make([]byte, 4), + start: start, + length: length, + postings: postings, + uvarintEncodeBuf: make([]byte, binary.MaxVarintLen64), + ctx: ctx, + } + + return prb +} + +func getInt32(r io.Reader, buf []byte) (uint32, error) { + read, err := io.ReadFull(r, buf) + if err != nil { + return 0, errors.Wrap(err, "reading") + } + if read != 4 { + return 0, fmt.Errorf("read got %d bytes instead of 4", read) + } + return binary.BigEndian.Uint32(buf), nil +} + +func (r *postingsReaderBuilder) Next() bool { + if r.ctx.Err() != nil { + r.e = r.ctx.Err() + return false + } + if r.repeatFor > 0 { + r.keyID = r.postings[r.pi-r.repeatFor].keyID + r.repeatFor-- + return true + } + if r.pi >= len(r.postings) { + return false + } + if r.Error() != nil { + return false + } + from := r.postings[r.pi].ptr.Start - r.start + + if from-r.lastOffset < 0 { + panic("would have skipped negative bytes") + } + + _, err := r.r.Discard(int(from - r.lastOffset)) + if err != nil { + r.e = err + return false + } + r.lastOffset += from - r.lastOffset + + postingsCount, err := getInt32(r.r, r.readBuf[:]) + if err != nil { + r.e = err + return false + } + r.lastOffset += 4 + + // Assume 1.25 bytes per compressed posting. + r.cur = make([]byte, 0, int(float64(postingsCount)*1.25)) + + prev := uint32(0) + + for i := 0; i < int(postingsCount); i++ { + posting, err := getInt32(r.r, r.readBuf[:]) + if err != nil { + r.e = err + return false + } + r.lastOffset += 4 + + uvarintSize := binary.PutUvarint(r.uvarintEncodeBuf, uint64(posting-prev)) + r.cur = append(r.cur, r.uvarintEncodeBuf[:uvarintSize]...) + prev = posting + } + r.numberOfPostingsInCur = uint64(postingsCount) + + r.keyID = r.postings[r.pi].keyID + r.pi++ + for { + if r.pi >= len(r.postings) { + break + } + + if r.postings[r.pi].ptr.Start == r.postings[r.pi-1].ptr.Start && + r.postings[r.pi].ptr.End == r.postings[r.pi-1].ptr.End { + r.repeatFor++ + r.pi++ + continue + } + + break + } + + return true +} + +func (r *postingsReaderBuilder) Error() error { + return r.e +} + +func (r *postingsReaderBuilder) AtDiffVarint() ([]byte, uint64, int) { + return r.cur, r.numberOfPostingsInCur, r.keyID +} diff --git a/pkg/storegateway/postings_codec.go b/pkg/storegateway/postings_codec.go new file mode 100644 index 0000000000..d9ceaa64ba --- /dev/null +++ b/pkg/storegateway/postings_codec.go @@ -0,0 +1,580 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "bytes" + "encoding/binary" + "fmt" + "hash/crc32" + "io" + + "github.com/golang/snappy" + "github.com/klauspost/compress/s2" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/encoding" + "github.com/prometheus/prometheus/tsdb/index" + extsnappy "github.com/thanos-io/thanos/pkg/extgrpc/snappy" + "github.com/thanos-io/thanos/pkg/pool" +) + +// This file implements encoding and decoding of postings using diff (or delta) + varint +// number encoding. On top of that, we apply Snappy compression. +// +// On its own, Snappy compressing raw postings doesn't really help, because there is no +// repetition in raw data. Using diff (delta) between postings entries makes values small, +// and Varint is very efficient at encoding small values (values < 128 are encoded as +// single byte, values < 16384 are encoded as two bytes). Diff + varint reduces postings size +// significantly (to about 20% of original), snappy then halves it to ~10% of the original. + +const ( + codecHeaderSnappy = "dvs" // As in "diff+varint+snappy". + codecHeaderStreamedSnappy = "dss" // As in "diffvarint+streamed snappy". +) + +func decodePostings(input []byte) (closeablePostings, error) { + var df func([]byte, bool) (closeablePostings, error) + + switch { + case isDiffVarintSnappyEncodedPostings(input): + df = diffVarintSnappyDecode + case isDiffVarintSnappyStreamedEncodedPostings(input): + df = diffVarintSnappyStreamedDecode + default: + return nil, fmt.Errorf("unrecognize postings format") + } + + return df(input, false) +} + +// isDiffVarintSnappyEncodedPostings returns true, if input looks like it has been encoded by diff+varint+snappy codec. +func isDiffVarintSnappyEncodedPostings(input []byte) bool { + return bytes.HasPrefix(input, []byte(codecHeaderSnappy)) +} + +// isDiffVarintSnappyStreamedEncodedPostings returns true, if input looks like it has been encoded by diff+varint+snappy streamed codec. +func isDiffVarintSnappyStreamedEncodedPostings(input []byte) bool { + return bytes.HasPrefix(input, []byte(codecHeaderStreamedSnappy)) +} + +// estimateSnappyStreamSize estimates the number of bytes +// needed for encoding length postings. Note that in reality +// the number of bytes needed could be much bigger if postings +// different by a lot. Practically, stddev=64 is used. +func estimateSnappyStreamSize(length int) int { + // Snappy stream writes data in chunks up to 65536 in size. + // The stream begins with bytes 0xff 0x06 0x00 0x00 's' 'N' 'a' 'P' 'p' 'Y'. + // Our encoded data also needs a header. + // Each encoded (or uncompressed) chunk needs tag (chunk type 1B + chunk len 3B) + checksum 4B. + + // Mark for encoded data. + ret := len(codecHeaderStreamedSnappy) + // Magic snappy stream start. + ret += 10 + + const maxBlockSize = 65536 + + length = 5 * length / 4 // estimate 1.25B per posting. + + blocks := length / maxBlockSize + + ret += blocks * snappy.MaxEncodedLen(maxBlockSize) + length -= blocks * maxBlockSize + if length > 0 { + ret += snappy.MaxEncodedLen(length) + } + + return ret +} + +func diffVarintSnappyStreamedEncode(p index.Postings, length int) ([]byte, error) { + compressedBuf := bytes.NewBuffer(make([]byte, 0, estimateSnappyStreamSize(length))) + if n, err := compressedBuf.WriteString(codecHeaderStreamedSnappy); err != nil { + return nil, fmt.Errorf("writing streamed snappy header") + } else if n != len(codecHeaderStreamedSnappy) { + return nil, fmt.Errorf("short-write streamed snappy header") + } + + uvarintEncodeBuf := make([]byte, binary.MaxVarintLen64) + + sw, err := extsnappy.Compressor.Compress(compressedBuf) + if err != nil { + return nil, fmt.Errorf("creating snappy compressor: %w", err) + } + + prev := storage.SeriesRef(0) + for p.Next() { + v := p.At() + if v < prev { + return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) + } + + uvarintSize := binary.PutUvarint(uvarintEncodeBuf, uint64(v-prev)) + if written, err := sw.Write(uvarintEncodeBuf[:uvarintSize]); err != nil { + return nil, errors.Wrap(err, "writing uvarint encoded byte") + } else if written != uvarintSize { + return nil, errors.Wrap(err, "short-write for uvarint encoded byte") + } + + prev = v + } + if p.Err() != nil { + return nil, p.Err() + } + if err := sw.Close(); err != nil { + return nil, errors.Wrap(err, "closing snappy stream writer") + } + + return compressedBuf.Bytes(), nil +} + +func diffVarintSnappyStreamedDecode(input []byte, disablePooling bool) (closeablePostings, error) { + if !isDiffVarintSnappyStreamedEncodedPostings(input) { + return nil, errors.New("header not found") + } + + return newStreamedDiffVarintPostings(input[len(codecHeaderStreamedSnappy):], disablePooling) +} + +type streamedDiffVarintPostings struct { + curSeries storage.SeriesRef + + err error + input, buf []byte + maximumDecodedLen int + + db *encoding.Decbuf + + readSnappyIdentifier bool + disablePooling bool +} + +const ( + chunkTypeCompressedData = 0x00 + chunkTypeUncompressedData = 0x01 + chunkTypeStreamIdentifier = 0xff + chunkTypePadding = 0xfe + checksumSize = 4 +) + +func maximumDecodedLenSnappyStreamed(in []byte) (int, error) { + maxDecodedLen := -1 + + for len(in) > 0 { + // Chunk type. + chunkType := in[0] + in = in[1:] + chunkLen := int(in[0]) | int(in[1])<<8 | int(in[2])<<16 + in = in[3:] + + switch chunkType { + case chunkTypeCompressedData: + bl := in[:chunkLen] + // NOTE: checksum will be checked later on. + decodedLen, err := s2.DecodedLen(bl[checksumSize:]) + if err != nil { + return 0, err + } + if decodedLen > maxDecodedLen { + maxDecodedLen = decodedLen + } + case chunkTypeUncompressedData: + // NOTE: checksum will be checked later on. + n := chunkLen - checksumSize + if n > maxDecodedLen { + maxDecodedLen = n + } + } + in = in[chunkLen:] + } + return maxDecodedLen, nil +} + +var decodedBufPool = pool.MustNewBucketedBytes(1024, 65536, 2, 0) + +func newStreamedDiffVarintPostings(input []byte, disablePooling bool) (closeablePostings, error) { + // We can't use the regular s2.Reader because it assumes a stream. + // We already everything in memory so let's avoid copying. + // Algorithm: + // 1. Step through all chunks all get maximum decoded len. + // 2. Read into decoded step by step. For decoding call s2.Decode(r.decoded, buf). + maximumDecodedLen, err := maximumDecodedLenSnappyStreamed(input) + if err != nil { + return nil, err + } + + return &streamedDiffVarintPostings{ + input: input, + maximumDecodedLen: maximumDecodedLen, + db: &encoding.Decbuf{}, + disablePooling: disablePooling, + }, nil +} + +func (it *streamedDiffVarintPostings) close() { + if it.buf == nil { + return + } + if it.disablePooling { + return + } + decodedBufPool.Put(&it.buf) +} + +func (it *streamedDiffVarintPostings) At() storage.SeriesRef { + return it.curSeries +} + +func (it *streamedDiffVarintPostings) readNextChunk(remainder []byte) bool { + // Normal EOF. + if len(it.input) == 0 { + return false + } + + // Read next chunk into it.db.B. + chunkType := it.input[0] + it.input = it.input[1:] + + if len(it.input) < 3 { + it.err = io.ErrUnexpectedEOF + return false + } + + chunkLen := int(it.input[0]) | int(it.input[1])<<8 | int(it.input[2])<<16 + it.input = it.input[3:] + + switch chunkType { + case chunkTypeStreamIdentifier: + const magicBody = "sNaPpY" + if chunkLen != len(magicBody) { + it.err = fmt.Errorf("corrupted identifier") + return false + } + if string(it.input[:len(magicBody)]) != magicBody { + it.err = fmt.Errorf("got bad identifier %s", string(it.input[:6])) + return false + } + it.input = it.input[6:] + it.readSnappyIdentifier = true + return it.readNextChunk(nil) + case chunkTypeCompressedData: + if !it.readSnappyIdentifier { + it.err = fmt.Errorf("missing magic snappy marker") + return false + } + if len(it.input) < 4 { + it.err = io.ErrUnexpectedEOF + return false + } + checksum := uint32(it.input[0]) | uint32(it.input[1])<<8 | uint32(it.input[2])<<16 | uint32(it.input[3])<<24 + if len(it.input) < chunkLen { + it.err = io.ErrUnexpectedEOF + return false + } + + if it.buf == nil { + if it.disablePooling { + it.buf = make([]byte, it.maximumDecodedLen) + } else { + b, err := decodedBufPool.Get(it.maximumDecodedLen) + if err != nil { + it.err = err + return false + } + it.buf = *b + } + } + + encodedBuf := it.input[:chunkLen] + + // NOTE(GiedriusS): we can probably optimize this better but this should be rare enough + // and not cause any problems. + if len(remainder) > 0 { + remainderCopy := make([]byte, 0, len(remainder)) + remainderCopy = append(remainderCopy, remainder...) + remainder = remainderCopy + } + decoded, err := s2.Decode(it.buf, encodedBuf[checksumSize:]) + if err != nil { + it.err = err + return false + } + if crc(decoded) != checksum { + it.err = fmt.Errorf("mismatched checksum (got %v, expected %v)", crc(decoded), checksum) + return false + } + if len(remainder) > 0 { + it.db.B = append(remainder, decoded...) + } else { + it.db.B = decoded + } + case chunkTypeUncompressedData: + if !it.readSnappyIdentifier { + it.err = fmt.Errorf("missing magic snappy marker") + return false + } + if len(it.input) < 4 { + it.err = io.ErrUnexpectedEOF + return false + } + checksum := uint32(it.input[0]) | uint32(it.input[1])<<8 | uint32(it.input[2])<<16 | uint32(it.input[3])<<24 + if len(it.input) < chunkLen { + it.err = io.ErrUnexpectedEOF + return false + } + uncompressedData := it.input[checksumSize:chunkLen] + if crc(uncompressedData) != checksum { + it.err = fmt.Errorf("mismatched checksum (got %v, expected %v)", crc(uncompressedData), checksum) + return false + } + + // NOTE(GiedriusS): we can probably optimize this better but this should be rare enough + // and not cause any problems. + if len(remainder) > 0 { + remainderCopy := make([]byte, 0, len(remainder)) + remainderCopy = append(remainderCopy, remainder...) + remainder = remainderCopy + } + + if len(remainder) > 0 { + it.db.B = append(remainder, uncompressedData...) + } else { + it.db.B = uncompressedData + } + default: + if chunkType <= 0x7f { + it.err = fmt.Errorf("unsupported chunk type %v", chunkType) + return false + } + if chunkType > 0xfd { + it.err = fmt.Errorf("invalid chunk type %v", chunkType) + return false + } + } + it.input = it.input[chunkLen:] + + return true +} + +func (it *streamedDiffVarintPostings) Next() bool { + // Continue reading next chunks until there is at least binary.MaxVarintLen64. + // If we cannot add any more chunks then return false. + for { + val := it.db.Uvarint64() + if it.db.Err() != nil { + if !it.readNextChunk(it.db.B) { + return false + } + it.db.E = nil + continue + } + + it.curSeries = it.curSeries + storage.SeriesRef(val) + return true + } +} + +func (it *streamedDiffVarintPostings) Err() error { + return it.err +} + +func (it *streamedDiffVarintPostings) Seek(x storage.SeriesRef) bool { + if it.curSeries >= x { + return true + } + + // We cannot do any search due to how values are stored, + // so we simply advance until we find the right value. + for it.Next() { + if it.At() >= x { + return true + } + } + + return false +} + +// diffVarintSnappyEncode encodes postings into diff+varint representation, +// and applies snappy compression on the result. +// Returned byte slice starts with codecHeaderSnappy header. +// Length argument is expected number of postings, used for preallocating buffer. +// TODO(GiedriusS): remove for v1.0. +func diffVarintSnappyEncode(p index.Postings, length int) ([]byte, error) { + buf, err := diffVarintEncodeNoHeader(p, length) + if err != nil { + return nil, err + } + + // Make result buffer large enough to hold our header and compressed block. + result := make([]byte, len(codecHeaderSnappy)+snappy.MaxEncodedLen(len(buf))) + copy(result, codecHeaderSnappy) + + compressed := snappy.Encode(result[len(codecHeaderSnappy):], buf) + + // Slice result buffer based on compressed size. + result = result[:len(codecHeaderSnappy)+len(compressed)] + return result, nil +} + +// diffVarintEncodeNoHeader encodes postings into diff+varint representation. +// It doesn't add any header to the output bytes. +// Length argument is expected number of postings, used for preallocating buffer. +func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { + buf := encoding.Encbuf{} + + // This encoding uses around ~1 bytes per posting, but let's use + // conservative 1.25 bytes per posting to avoid extra allocations. + if length > 0 { + buf.B = make([]byte, 0, 5*length/4) + } + + prev := storage.SeriesRef(0) + for p.Next() { + v := p.At() + if v < prev { + return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) + } + + // This is the 'diff' part -- compute difference from previous value. + buf.PutUvarint64(uint64(v - prev)) + prev = v + } + if p.Err() != nil { + return nil, p.Err() + } + + return buf.B, nil +} + +// Creating 15 buckets from 1k to 32mb. +var snappyDecodePool = pool.MustNewBucketedBytes(1024, 32*1024*1024, 2, 0) + +type closeablePostings interface { + index.Postings + close() +} + +// alias returns true if given slices have the same both backing array. +// See: https://groups.google.com/g/golang-nuts/c/C6ufGl73Uzk. +func alias(x, y []byte) bool { + return cap(x) > 0 && cap(y) > 0 && &x[0:cap(x)][cap(x)-1] == &y[0:cap(y)][cap(y)-1] +} + +// TODO(GiedriusS): remove for v1.0. +func diffVarintSnappyDecode(input []byte, disablePooling bool) (closeablePostings, error) { + if !isDiffVarintSnappyEncodedPostings(input) { + return nil, errors.New("header not found") + } + + toFree := make([][]byte, 0, 2) + + var dstBuf []byte + if !disablePooling { + if len, err := s2.DecodedLen(input[len(codecHeaderSnappy):]); err == nil { + if decodeBuf, err := snappyDecodePool.Get(len); err == nil && decodeBuf != nil { + dstBuf = *decodeBuf + toFree = append(toFree, dstBuf) + } + } + } + + raw, err := s2.Decode(dstBuf, input[len(codecHeaderSnappy):]) + if err != nil { + return nil, errors.Wrap(err, "snappy decode") + } + + if !alias(raw, dstBuf) && !disablePooling { + toFree = append(toFree, raw) + } + + return newDiffVarintPostings(raw, toFree), nil +} + +func newDiffVarintPostings(input []byte, freeSlices [][]byte) *diffVarintPostings { + return &diffVarintPostings{freeSlices: freeSlices, buf: &encoding.Decbuf{B: input}} +} + +// diffVarintPostings is an implementation of index.Postings based on diff+varint encoded data. +type diffVarintPostings struct { + buf *encoding.Decbuf + cur storage.SeriesRef + freeSlices [][]byte +} + +func (it *diffVarintPostings) close() { + for i := range it.freeSlices { + snappyDecodePool.Put(&it.freeSlices[i]) + } +} + +func (it *diffVarintPostings) At() storage.SeriesRef { + return it.cur +} + +func (it *diffVarintPostings) Next() bool { + if it.buf.Err() != nil || it.buf.Len() == 0 { + return false + } + + val := it.buf.Uvarint64() + if it.buf.Err() != nil { + return false + } + + it.cur = it.cur + storage.SeriesRef(val) + return true +} + +func (it *diffVarintPostings) Seek(x storage.SeriesRef) bool { + if it.cur >= x { + return true + } + + // We cannot do any search due to how values are stored, + // so we simply advance until we find the right value. + for it.Next() { + if it.At() >= x { + return true + } + } + + return false +} + +func (it *diffVarintPostings) Err() error { + return it.buf.Err() +} + +func snappyStreamedEncode(postingsLength int, diffVarintPostings []byte) ([]byte, error) { + compressedBuf := bytes.NewBuffer(make([]byte, 0, estimateSnappyStreamSize(postingsLength))) + if n, err := compressedBuf.WriteString(codecHeaderStreamedSnappy); err != nil { + return nil, fmt.Errorf("writing streamed snappy header") + } else if n != len(codecHeaderStreamedSnappy) { + return nil, fmt.Errorf("short-write streamed snappy header") + } + + sw, err := extsnappy.Compressor.Compress(compressedBuf) + if err != nil { + return nil, fmt.Errorf("creating snappy compressor: %w", err) + } + _, err = sw.Write(diffVarintPostings) + if err != nil { + return nil, err + } + if err := sw.Close(); err != nil { + return nil, errors.Wrap(err, "closing snappy stream writer") + } + + return compressedBuf.Bytes(), nil +} + +var crcTable = crc32.MakeTable(crc32.Castagnoli) + +// crc implements the checksum specified in section 3 of +// https://github.com/google/snappy/blob/master/framing_format.txt +func crc(b []byte) uint32 { + c := crc32.Update(0, crcTable, b) + return c>>15 | c<<17 + 0xa282ead8 +} diff --git a/pkg/storegateway/proxy_heap.go b/pkg/storegateway/proxy_heap.go new file mode 100644 index 0000000000..032fbbf9b4 --- /dev/null +++ b/pkg/storegateway/proxy_heap.go @@ -0,0 +1,758 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storegateway + +import ( + "container/heap" + "context" + "io" + "sort" + "sync" + "time" + + "github.com/cespare/xxhash/v2" + "github.com/opentracing/opentracing-go" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + + "github.com/cortexproject/cortex/pkg/storegateway/storepb" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +type dedupResponseHeap struct { + h *ProxyResponseHeap + + bufferedSameSeries []*storepb.SeriesResponse + + bufferedResp []*storepb.SeriesResponse + buffRespI int + + prev *storepb.SeriesResponse + ok bool +} + +// NewDedupResponseHeap returns a wrapper around ProxyResponseHeap that merged duplicated series messages into one. +// It also deduplicates identical chunks identified by the same checksum from each series message. +func NewDedupResponseHeap(h *ProxyResponseHeap) *dedupResponseHeap { + ok := h.Next() + var prev *storepb.SeriesResponse + if ok { + prev = h.At() + } + return &dedupResponseHeap{ + h: h, + ok: ok, + prev: prev, + } +} + +func (d *dedupResponseHeap) Next() bool { + if d.buffRespI+1 < len(d.bufferedResp) { + d.buffRespI++ + return true + } + + if !d.ok && d.prev == nil { + return false + } + + d.buffRespI = 0 + d.bufferedResp = d.bufferedResp[:0] + d.bufferedSameSeries = d.bufferedSameSeries[:0] + + var s *storepb.SeriesResponse + for { + if d.prev == nil { + d.ok = d.h.Next() + if !d.ok { + if len(d.bufferedSameSeries) > 0 { + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + } + return len(d.bufferedResp) > 0 + } + s = d.h.At() + } else { + s = d.prev + d.prev = nil + } + + if s.GetSeries() == nil { + d.bufferedResp = append(d.bufferedResp, s) + continue + } + + if len(d.bufferedSameSeries) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue + } + + lbls := d.bufferedSameSeries[0].GetSeries().Labels + atLbls := s.GetSeries().Labels + + if labels.Compare(labelpb.ZLabelsToPromLabels(lbls), labelpb.ZLabelsToPromLabels(atLbls)) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue + } + + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + d.prev = s + + return true + } +} + +func chainSeriesAndRemIdenticalChunks(series []*storepb.SeriesResponse) *storepb.SeriesResponse { + chunkDedupMap := map[uint64]*typespb.AggrChunk{} + + for _, s := range series { + for _, chk := range s.GetSeries().Chunks { + for _, field := range []*typespb.Chunk{ + chk.Raw, chk.Count, chk.Max, chk.Min, chk.Sum, chk.Counter, + } { + if field == nil { + continue + } + hash := field.Hash + if hash == 0 { + hash = xxhash.Sum64(field.Data) + } + + if _, ok := chunkDedupMap[hash]; !ok { + chk := chk + chunkDedupMap[hash] = &chk + break + } + } + } + } + + // If no chunks were requested. + if len(chunkDedupMap) == 0 { + return series[0] + } + + finalChunks := make([]typespb.AggrChunk, 0, len(chunkDedupMap)) + for _, chk := range chunkDedupMap { + finalChunks = append(finalChunks, *chk) + } + + sort.Slice(finalChunks, func(i, j int) bool { + return finalChunks[i].Compare(finalChunks[j]) > 0 + }) + + return storepb.NewSeriesResponse(&typespb.Series{ + Labels: series[0].GetSeries().Labels, + Chunks: finalChunks, + }) +} + +func (d *dedupResponseHeap) At() *storepb.SeriesResponse { + return d.bufferedResp[d.buffRespI] +} + +// ProxyResponseHeap is a heap for storepb.SeriesSets. +// It performs k-way merge between all of those sets. +// TODO(GiedriusS): can be improved with a tournament tree. +// This is O(n*logk) but can be Theta(n*logk). However, +// tournament trees need n-1 auxiliary nodes so there +// might not be much of a difference. +type ProxyResponseHeap struct { + nodes []ProxyResponseHeapNode +} + +func (h *ProxyResponseHeap) Less(i, j int) bool { + iResp := h.nodes[i].rs.At() + jResp := h.nodes[j].rs.At() + + if iResp.GetSeries() != nil && jResp.GetSeries() != nil { + iLbls := labelpb.ZLabelsToPromLabels(iResp.GetSeries().Labels) + jLbls := labelpb.ZLabelsToPromLabels(jResp.GetSeries().Labels) + + return labels.Compare(iLbls, jLbls) < 0 + } else if iResp.GetSeries() == nil && jResp.GetSeries() != nil { + return true + } else if iResp.GetSeries() != nil && jResp.GetSeries() == nil { + return false + } + + // If it is not a series then the order does not matter. What matters + // is that we get different types of responses one after another. + return false +} + +func (h *ProxyResponseHeap) Len() int { + return len(h.nodes) +} + +func (h *ProxyResponseHeap) Swap(i, j int) { + h.nodes[i], h.nodes[j] = h.nodes[j], h.nodes[i] +} + +func (h *ProxyResponseHeap) Push(x interface{}) { + h.nodes = append(h.nodes, x.(ProxyResponseHeapNode)) +} + +func (h *ProxyResponseHeap) Pop() (v interface{}) { + h.nodes, v = h.nodes[:h.Len()-1], h.nodes[h.Len()-1] + return +} + +func (h *ProxyResponseHeap) Empty() bool { + return h.Len() == 0 +} + +func (h *ProxyResponseHeap) Min() *ProxyResponseHeapNode { + return &h.nodes[0] +} + +type ProxyResponseHeapNode struct { + rs respSet +} + +// NewProxyResponseHeap returns heap that k-way merge series together. +// It's agnostic to duplicates and overlaps, it forwards all duplicated series in random order. +func NewProxyResponseHeap(seriesSets ...respSet) *ProxyResponseHeap { + ret := ProxyResponseHeap{ + nodes: make([]ProxyResponseHeapNode, 0, len(seriesSets)), + } + + for _, ss := range seriesSets { + if ss.Empty() { + continue + } + ss := ss + ret.Push(ProxyResponseHeapNode{rs: ss}) + } + + heap.Init(&ret) + + return &ret +} + +func (h *ProxyResponseHeap) Next() bool { + return !h.Empty() +} + +func (h *ProxyResponseHeap) At() *storepb.SeriesResponse { + min := h.Min().rs + + atResp := min.At() + + if min.Next() { + heap.Fix(h, 0) + } else { + heap.Remove(h, 0) + } + + return atResp +} + +func (l *lazyRespSet) StoreID() string { + return l.storeName +} + +func (l *lazyRespSet) Labelset() string { + return labelpb.PromLabelSetsToString(l.storeLabelSets) +} + +func (l *lazyRespSet) StoreLabels() map[string]struct{} { + return l.storeLabels +} + +// lazyRespSet is a lazy storepb.SeriesSet that buffers +// everything as fast as possible while at the same it permits +// reading response-by-response. It blocks if there is no data +// in Next(). +type lazyRespSet struct { + // Generic parameters. + span opentracing.Span + cl storepb.Store_SeriesClient + closeSeries context.CancelFunc + storeName string + storeLabelSets []labels.Labels + storeLabels map[string]struct{} + frameTimeout time.Duration + ctx context.Context + + // Internal bookkeeping. + dataOrFinishEvent *sync.Cond + bufferedResponses []*storepb.SeriesResponse + bufferedResponsesMtx *sync.Mutex + lastResp *storepb.SeriesResponse + + noMoreData bool + initialized bool + + shardMatcher *storepb.ShardMatcher +} + +func (l *lazyRespSet) Empty() bool { + l.bufferedResponsesMtx.Lock() + defer l.bufferedResponsesMtx.Unlock() + + // NOTE(GiedriusS): need to wait here for at least one + // response so that we could build the heap properly. + if l.noMoreData && len(l.bufferedResponses) == 0 { + return true + } + + for len(l.bufferedResponses) == 0 { + l.dataOrFinishEvent.Wait() + if l.noMoreData && len(l.bufferedResponses) == 0 { + break + } + } + + return len(l.bufferedResponses) == 0 && l.noMoreData +} + +// Next either blocks until more data is available or reads +// the next response. If it is not lazy then it waits for everything +// to finish. +func (l *lazyRespSet) Next() bool { + l.bufferedResponsesMtx.Lock() + defer l.bufferedResponsesMtx.Unlock() + + if l.noMoreData && len(l.bufferedResponses) == 0 { + l.lastResp = nil + + return false + } + + for len(l.bufferedResponses) == 0 { + l.dataOrFinishEvent.Wait() + if l.noMoreData && len(l.bufferedResponses) == 0 { + break + } + } + + if len(l.bufferedResponses) > 0 { + l.lastResp = l.bufferedResponses[0] + l.bufferedResponses = l.bufferedResponses[1:] + return true + } + + l.lastResp = nil + return false +} + +func (l *lazyRespSet) At() *storepb.SeriesResponse { + // We need to wait for at least one response so that we would be able to properly build the heap. + if !l.initialized { + l.Next() + l.initialized = true + return l.lastResp + } + + // Next() was called previously. + return l.lastResp +} + +func newLazyRespSet( + ctx context.Context, + span opentracing.Span, + frameTimeout time.Duration, + storeName string, + storeLabelSets []labels.Labels, + closeSeries context.CancelFunc, + cl storepb.Store_SeriesClient, + shardMatcher *storepb.ShardMatcher, + applySharding bool, + emptyStreamResponses prometheus.Counter, +) respSet { + bufferedResponses := []*storepb.SeriesResponse{} + bufferedResponsesMtx := &sync.Mutex{} + dataAvailable := sync.NewCond(bufferedResponsesMtx) + + respSet := &lazyRespSet{ + frameTimeout: frameTimeout, + cl: cl, + storeName: storeName, + storeLabelSets: storeLabelSets, + closeSeries: closeSeries, + span: span, + ctx: ctx, + dataOrFinishEvent: dataAvailable, + bufferedResponsesMtx: bufferedResponsesMtx, + bufferedResponses: bufferedResponses, + shardMatcher: shardMatcher, + } + respSet.storeLabels = make(map[string]struct{}) + for _, ls := range storeLabelSets { + for _, l := range ls { + respSet.storeLabels[l.Name] = struct{}{} + } + } + + go func(st string, l *lazyRespSet) { + bytesProcessed := 0 + seriesStats := &storepb.SeriesStatsCounter{} + + defer func() { + l.span.SetTag("processed.series", seriesStats.Series) + l.span.SetTag("processed.chunks", seriesStats.Chunks) + l.span.SetTag("processed.samples", seriesStats.Samples) + l.span.SetTag("processed.bytes", bytesProcessed) + l.span.Finish() + }() + + numResponses := 0 + defer func() { + if numResponses == 0 { + emptyStreamResponses.Inc() + } + }() + + handleRecvResponse := func(t *time.Timer) bool { + if t != nil { + defer t.Reset(frameTimeout) + } + + select { + case <-l.ctx.Done(): + err := errors.Wrapf(l.ctx.Err(), "failed to receive any data from %s", st) + l.span.SetTag("err", err.Error()) + + l.bufferedResponsesMtx.Lock() + l.bufferedResponses = append(l.bufferedResponses, storepb.NewWarnSeriesResponse(err)) + l.noMoreData = true + l.dataOrFinishEvent.Signal() + l.bufferedResponsesMtx.Unlock() + return false + default: + resp, err := cl.Recv() + if err == io.EOF { + l.bufferedResponsesMtx.Lock() + l.noMoreData = true + l.dataOrFinishEvent.Signal() + l.bufferedResponsesMtx.Unlock() + return false + } + + if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. + var rerr error + if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { + // Most likely the per-Recv timeout has been reached. + // There's a small race between canceling and the Recv() + // but this is most likely true. + rerr = errors.Wrapf(err, "failed to receive any data in %s from %s", l.frameTimeout, st) + } else { + rerr = errors.Wrapf(err, "receive series from %s", st) + } + + l.span.SetTag("err", rerr.Error()) + + l.bufferedResponsesMtx.Lock() + l.bufferedResponses = append(l.bufferedResponses, storepb.NewWarnSeriesResponse(rerr)) + l.noMoreData = true + l.dataOrFinishEvent.Signal() + l.bufferedResponsesMtx.Unlock() + return false + } + + numResponses++ + bytesProcessed += resp.Size() + + if resp.GetSeries() != nil && applySharding && !shardMatcher.MatchesZLabels(resp.GetSeries().Labels) { + return true + } + + if resp.GetSeries() != nil { + seriesStats.Count(resp.GetSeries()) + } + + l.bufferedResponsesMtx.Lock() + l.bufferedResponses = append(l.bufferedResponses, resp) + l.dataOrFinishEvent.Signal() + l.bufferedResponsesMtx.Unlock() + return true + } + } + + var t *time.Timer + if frameTimeout > 0 { + t = time.AfterFunc(frameTimeout, closeSeries) + defer t.Stop() + } + for { + if !handleRecvResponse(t) { + return + } + } + }(storeName, respSet) + + return respSet +} + +// RetrievalStrategy stores what kind of retrieval strategy +// shall be used for the async response set. +type RetrievalStrategy string + +const ( + // LazyRetrieval allows readers (e.g. PromQL engine) to use (stream) data as soon as possible. + LazyRetrieval RetrievalStrategy = "lazy" + // EagerRetrieval is optimized to read all into internal buffer before returning to readers (e.g. PromQL engine). + // This currently preferred because: + // * Both PromQL engines (old and new) want all series ASAP to make decisions. + // * Querier buffers all responses when using StoreAPI internally. + EagerRetrieval RetrievalStrategy = "eager" +) + +func (l *lazyRespSet) Close() { + l.bufferedResponsesMtx.Lock() + defer l.bufferedResponsesMtx.Unlock() + + l.closeSeries() + l.noMoreData = true + l.dataOrFinishEvent.Signal() + + l.shardMatcher.Close() +} + +// eagerRespSet is a SeriesSet that blocks until all data is retrieved from +// the StoreAPI. +// NOTE(bwplotka): It also resorts the series (and emits warning) if the client.SupportsWithoutReplicaLabels() is false. +type eagerRespSet struct { + // Generic parameters. + span opentracing.Span + cl storepb.Store_SeriesClient + ctx context.Context + + closeSeries context.CancelFunc + frameTimeout time.Duration + + shardMatcher *storepb.ShardMatcher + removeLabels map[string]struct{} + + storeName string + storeLabels map[string]struct{} + storeLabelSets []labels.Labels + + // Internal bookkeeping. + bufferedResponses []*storepb.SeriesResponse + wg *sync.WaitGroup + i int +} + +func newEagerRespSet( + ctx context.Context, + span opentracing.Span, + frameTimeout time.Duration, + storeName string, + storeLabelSets []labels.Labels, + closeSeries context.CancelFunc, + cl storepb.Store_SeriesClient, + shardMatcher *storepb.ShardMatcher, + applySharding bool, + emptyStreamResponses prometheus.Counter, + removeLabels map[string]struct{}, +) respSet { + ret := &eagerRespSet{ + span: span, + closeSeries: closeSeries, + cl: cl, + frameTimeout: frameTimeout, + ctx: ctx, + bufferedResponses: []*storepb.SeriesResponse{}, + wg: &sync.WaitGroup{}, + shardMatcher: shardMatcher, + removeLabels: removeLabels, + storeName: storeName, + storeLabelSets: storeLabelSets, + } + ret.storeLabels = make(map[string]struct{}) + for _, ls := range storeLabelSets { + for _, l := range ls { + ret.storeLabels[l.Name] = struct{}{} + } + } + + ret.wg.Add(1) + + // Start a goroutine and immediately buffer everything. + go func(l *eagerRespSet) { + seriesStats := &storepb.SeriesStatsCounter{} + bytesProcessed := 0 + + defer func() { + l.span.SetTag("processed.series", seriesStats.Series) + l.span.SetTag("processed.chunks", seriesStats.Chunks) + l.span.SetTag("processed.samples", seriesStats.Samples) + l.span.SetTag("processed.bytes", bytesProcessed) + l.span.Finish() + ret.wg.Done() + }() + + numResponses := 0 + defer func() { + if numResponses == 0 { + emptyStreamResponses.Inc() + } + }() + + // TODO(bwplotka): Consider improving readability by getting rid of anonymous functions and merging eager and + // lazyResponse into one struct. + handleRecvResponse := func(t *time.Timer) bool { + if t != nil { + defer t.Reset(frameTimeout) + } + + select { + case <-l.ctx.Done(): + err := errors.Wrapf(l.ctx.Err(), "failed to receive any data from %s", storeName) + l.bufferedResponses = append(l.bufferedResponses, storepb.NewWarnSeriesResponse(err)) + l.span.SetTag("err", err.Error()) + return false + default: + resp, err := cl.Recv() + if err == io.EOF { + return false + } + if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. + var rerr error + if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { + // Most likely the per-Recv timeout has been reached. + // There's a small race between canceling and the Recv() + // but this is most likely true. + rerr = errors.Wrapf(err, "failed to receive any data in %s from %s", l.frameTimeout, storeName) + } else { + rerr = errors.Wrapf(err, "receive series from %s", storeName) + } + l.bufferedResponses = append(l.bufferedResponses, storepb.NewWarnSeriesResponse(rerr)) + l.span.SetTag("err", rerr.Error()) + return false + } + + numResponses++ + bytesProcessed += resp.Size() + + if resp.GetSeries() != nil && applySharding && !shardMatcher.MatchesZLabels(resp.GetSeries().Labels) { + return true + } + + if resp.GetSeries() != nil { + seriesStats.Count(resp.GetSeries()) + } + + l.bufferedResponses = append(l.bufferedResponses, resp) + return true + } + } + var t *time.Timer + if frameTimeout > 0 { + t = time.AfterFunc(frameTimeout, closeSeries) + defer t.Stop() + } + + for { + if !handleRecvResponse(t) { + break + } + } + + // This should be used only for stores that does not support doing this on server side. + // See docs/proposals-accepted/20221129-avoid-global-sort.md for details. + // NOTE. Client is not guaranteed to give a sorted response when extLset is added + // Generally we need to resort here. + sortWithoutLabels(l.bufferedResponses, l.removeLabels) + + }(ret) + + return ret +} + +func rmLabels(l labels.Labels, labelsToRemove map[string]struct{}) labels.Labels { + for i := 0; i < len(l); i++ { + if _, ok := labelsToRemove[l[i].Name]; !ok { + continue + } + l = append(l[:i], l[i+1:]...) + i-- + } + return l +} + +// sortWithoutLabels removes given labels from series and re-sorts the series responses that the same +// series with different labels are coming right after each other. Other types of responses are moved to front. +func sortWithoutLabels(set []*storepb.SeriesResponse, labelsToRemove map[string]struct{}) { + for _, s := range set { + ser := s.GetSeries() + if ser == nil { + continue + } + + if len(labelsToRemove) > 0 { + ser.Labels = labelpb.ZLabelsFromPromLabels(rmLabels(labelpb.ZLabelsToPromLabels(ser.Labels), labelsToRemove)) + } + } + + // With the re-ordered label sets, re-sorting all series aligns the same series + // from different replicas sequentially. + sort.Slice(set, func(i, j int) bool { + si := set[i].GetSeries() + if si == nil { + return true + } + sj := set[j].GetSeries() + if sj == nil { + return false + } + return labels.Compare(labelpb.ZLabelsToPromLabels(si.Labels), labelpb.ZLabelsToPromLabels(sj.Labels)) < 0 + }) +} + +func (l *eagerRespSet) Close() { + l.closeSeries() + l.shardMatcher.Close() +} + +func (l *eagerRespSet) At() *storepb.SeriesResponse { + l.wg.Wait() + + if len(l.bufferedResponses) == 0 { + return nil + } + + return l.bufferedResponses[l.i] +} + +func (l *eagerRespSet) Next() bool { + l.wg.Wait() + + l.i++ + + return l.i < len(l.bufferedResponses) +} + +func (l *eagerRespSet) Empty() bool { + l.wg.Wait() + + return len(l.bufferedResponses) == 0 +} + +func (l *eagerRespSet) StoreID() string { + return l.storeName +} + +func (l *eagerRespSet) Labelset() string { + return labelpb.PromLabelSetsToString(l.storeLabelSets) +} + +func (l *eagerRespSet) StoreLabels() map[string]struct{} { + return l.storeLabels +} + +type respSet interface { + Close() + At() *storepb.SeriesResponse + Next() bool + StoreID() string + Labelset() string + StoreLabels() map[string]struct{} + Empty() bool +} diff --git a/pkg/storegateway/sorted_matchers.go b/pkg/storegateway/sorted_matchers.go new file mode 100644 index 0000000000..38a89a5749 --- /dev/null +++ b/pkg/storegateway/sorted_matchers.go @@ -0,0 +1,23 @@ +package storegateway + +import ( + "sort" + + "github.com/prometheus/prometheus/model/labels" +) + +type sortedMatchers []*labels.Matcher + +func newSortedMatchers(matchers []*labels.Matcher) sortedMatchers { + sort.Slice(matchers, func(i, j int) bool { + if matchers[i].Type == matchers[j].Type { + if matchers[i].Name == matchers[j].Name { + return matchers[i].Value < matchers[j].Value + } + return matchers[i].Name < matchers[j].Name + } + return matchers[i].Type < matchers[j].Type + }) + + return matchers +} diff --git a/pkg/storegateway/storegatewaypb/gateway.pb.go b/pkg/storegateway/storegatewaypb/gateway.pb.go index fa5913faf4..81d2d870f5 100644 --- a/pkg/storegateway/storegatewaypb/gateway.pb.go +++ b/pkg/storegateway/storegatewaypb/gateway.pb.go @@ -6,8 +6,9 @@ package storegatewaypb import ( context "context" fmt "fmt" + storepb "github.com/cortexproject/cortex/pkg/storegateway/storepb" + _ "github.com/cortexproject/cortex/pkg/storegateway/typespb" proto "github.com/gogo/protobuf/proto" - storepb "github.com/thanos-io/thanos/pkg/store/storepb" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" @@ -28,24 +29,27 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package func init() { proto.RegisterFile("gateway.proto", fileDescriptor_f1a937782ebbded5) } var fileDescriptor_f1a937782ebbded5 = []byte{ - // 257 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x4d, 0x4f, 0x2c, 0x49, - 0x2d, 0x4f, 0xac, 0xd4, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x84, 0x72, 0x0b, 0x92, 0xa4, - 0xcc, 0xd3, 0x33, 0x4b, 0x32, 0x4a, 0x93, 0xf4, 0x92, 0xf3, 0x73, 0xf5, 0x4b, 0x32, 0x12, 0xf3, - 0xf2, 0x8b, 0x75, 0x33, 0xf3, 0xa1, 0x2c, 0xfd, 0x82, 0xec, 0x74, 0xfd, 0xe2, 0x92, 0xfc, 0xa2, - 0x54, 0x08, 0x59, 0x90, 0xa4, 0x5f, 0x54, 0x90, 0x0c, 0x31, 0xc3, 0xe8, 0x1a, 0x23, 0x17, 0x4f, - 0x30, 0x48, 0xd4, 0x1d, 0x62, 0x96, 0x90, 0x25, 0x17, 0x5b, 0x70, 0x6a, 0x51, 0x66, 0x6a, 0xb1, - 0x90, 0xa8, 0x1e, 0x44, 0xbf, 0x1e, 0x84, 0x1f, 0x94, 0x5a, 0x58, 0x9a, 0x5a, 0x5c, 0x22, 0x25, - 0x86, 0x2e, 0x5c, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0x6a, 0xc0, 0x28, 0xe4, 0xcc, 0xc5, 0xe5, 0x93, - 0x98, 0x94, 0x9a, 0xe3, 0x97, 0x98, 0x9b, 0x5a, 0x2c, 0x24, 0x09, 0x53, 0x87, 0x10, 0x83, 0x19, - 0x21, 0x85, 0x4d, 0x0a, 0x62, 0x8c, 0x90, 0x1b, 0x17, 0x37, 0x58, 0x34, 0x2c, 0x31, 0xa7, 0x34, - 0xb5, 0x58, 0x08, 0x55, 0x29, 0x44, 0x10, 0x66, 0x8c, 0x34, 0x56, 0x39, 0x88, 0x39, 0x4e, 0x2e, - 0x17, 0x1e, 0xca, 0x31, 0xdc, 0x78, 0x28, 0xc7, 0xf0, 0xe1, 0xa1, 0x1c, 0x63, 0xc3, 0x23, 0x39, - 0xc6, 0x15, 0x8f, 0xe4, 0x18, 0x4f, 0x3c, 0x92, 0x63, 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc1, 0x23, - 0x39, 0xc6, 0x17, 0x8f, 0xe4, 0x18, 0x3e, 0x3c, 0x92, 0x63, 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, - 0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, 0xf8, 0xc0, 0x21, 0x04, 0x0f, 0xd7, 0x24, 0x36, - 0x70, 0x28, 0x19, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x1b, 0xec, 0xe6, 0x0a, 0x7a, 0x01, 0x00, - 0x00, + // 320 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x91, 0xbd, 0x4e, 0xf3, 0x30, + 0x14, 0x86, 0xe3, 0x6f, 0xa8, 0xf4, 0x19, 0xca, 0xe0, 0x01, 0xa4, 0x80, 0xce, 0xcc, 0x94, 0x54, + 0x65, 0x44, 0x2c, 0xa5, 0x08, 0x06, 0xd4, 0x81, 0x4a, 0x0c, 0x6c, 0x71, 0x74, 0xd4, 0x96, 0xfe, + 0xd8, 0xd8, 0x8e, 0xa0, 0x1b, 0x97, 0xc0, 0xca, 0x1d, 0x70, 0x29, 0x8c, 0x1d, 0x3b, 0x52, 0x77, + 0x61, 0xec, 0x25, 0xa0, 0xf6, 0x24, 0xa5, 0xd0, 0x2c, 0x4c, 0xd1, 0x79, 0xfc, 0xbe, 0x4f, 0x64, + 0x1f, 0x5e, 0xed, 0x24, 0x0e, 0x1f, 0x93, 0x71, 0xa4, 0x8d, 0x72, 0x4a, 0xfc, 0xcf, 0x47, 0x2d, + 0xc3, 0x46, 0xa7, 0xe7, 0xba, 0x99, 0x8c, 0x52, 0x35, 0x8c, 0x53, 0x65, 0x1c, 0x3e, 0x69, 0xa3, + 0xee, 0x31, 0x75, 0xf9, 0x14, 0xeb, 0x7e, 0x27, 0xb6, 0x4e, 0x19, 0xcc, 0x3b, 0x34, 0x68, 0x19, + 0x1b, 0x9d, 0x92, 0x2e, 0x6c, 0xfe, 0xd9, 0xe1, 0xc6, 0x1a, 0xad, 0x96, 0xf4, 0x25, 0x4b, 0xfd, + 0xf5, 0x1f, 0xdf, 0x6d, 0x2f, 0x43, 0x97, 0x14, 0x12, 0xa7, 0xbc, 0xd2, 0x46, 0xd3, 0x43, 0x2b, + 0xf6, 0xa3, 0xfc, 0xa7, 0x11, 0x81, 0x1b, 0x7c, 0xc8, 0xd0, 0xba, 0xf0, 0x60, 0x8b, 0x5b, 0xad, + 0x46, 0x16, 0x6b, 0x8c, 0xca, 0x03, 0x4c, 0xdd, 0x8f, 0xf2, 0x12, 0x94, 0x95, 0x89, 0xaf, 0xcb, + 0x17, 0x9c, 0x5f, 0x27, 0x12, 0x07, 0xad, 0x64, 0x88, 0x56, 0x84, 0xeb, 0xe0, 0x37, 0x2c, 0x24, + 0x87, 0xa5, 0x67, 0x24, 0x12, 0x57, 0x7c, 0x67, 0x45, 0x6f, 0x93, 0x41, 0x86, 0x56, 0xfc, 0xca, + 0x12, 0x2d, 0x44, 0x47, 0xe5, 0x87, 0x64, 0xaa, 0xb7, 0x78, 0xf5, 0xbc, 0x9b, 0x8d, 0xfa, 0xb6, + 0x78, 0x9b, 0x33, 0x5e, 0x21, 0xb0, 0x71, 0x3d, 0x02, 0xdb, 0xd7, 0x2b, 0x38, 0xb9, 0x8e, 0x59, + 0x8d, 0x35, 0x9a, 0x93, 0x19, 0x04, 0xd3, 0x19, 0x04, 0x8b, 0x19, 0xb0, 0x67, 0x0f, 0xec, 0xcd, + 0x03, 0x7b, 0xf7, 0xc0, 0x26, 0x1e, 0xd8, 0x87, 0x07, 0xf6, 0xe9, 0x21, 0x58, 0x78, 0x60, 0x2f, + 0x73, 0x08, 0x26, 0x73, 0x08, 0xa6, 0x73, 0x08, 0xee, 0xf6, 0x36, 0x77, 0xa8, 0xa5, 0xac, 0xac, + 0x16, 0x77, 0xf2, 0x15, 0x00, 0x00, 0xff, 0xff, 0x6b, 0x8e, 0xb8, 0x79, 0x5e, 0x02, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -68,6 +72,7 @@ type StoreGatewayClient interface { // // Series are sorted. Series(ctx context.Context, in *storepb.SeriesRequest, opts ...grpc.CallOption) (StoreGateway_SeriesClient, error) + Select(ctx context.Context, in *storepb.SelectRequest, opts ...grpc.CallOption) (StoreGateway_SelectClient, error) // LabelNames returns all label names that is available. LabelNames(ctx context.Context, in *storepb.LabelNamesRequest, opts ...grpc.CallOption) (*storepb.LabelNamesResponse, error) // LabelValues returns all label values for given label name. @@ -114,6 +119,38 @@ func (x *storeGatewaySeriesClient) Recv() (*storepb.SeriesResponse, error) { return m, nil } +func (c *storeGatewayClient) Select(ctx context.Context, in *storepb.SelectRequest, opts ...grpc.CallOption) (StoreGateway_SelectClient, error) { + stream, err := c.cc.NewStream(ctx, &_StoreGateway_serviceDesc.Streams[1], "/gatewaypb.StoreGateway/Select", opts...) + if err != nil { + return nil, err + } + x := &storeGatewaySelectClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type StoreGateway_SelectClient interface { + Recv() (*storepb.SelectResponse, error) + grpc.ClientStream +} + +type storeGatewaySelectClient struct { + grpc.ClientStream +} + +func (x *storeGatewaySelectClient) Recv() (*storepb.SelectResponse, error) { + m := new(storepb.SelectResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + func (c *storeGatewayClient) LabelNames(ctx context.Context, in *storepb.LabelNamesRequest, opts ...grpc.CallOption) (*storepb.LabelNamesResponse, error) { out := new(storepb.LabelNamesResponse) err := c.cc.Invoke(ctx, "/gatewaypb.StoreGateway/LabelNames", in, out, opts...) @@ -142,6 +179,7 @@ type StoreGatewayServer interface { // // Series are sorted. Series(*storepb.SeriesRequest, StoreGateway_SeriesServer) error + Select(*storepb.SelectRequest, StoreGateway_SelectServer) error // LabelNames returns all label names that is available. LabelNames(context.Context, *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) // LabelValues returns all label values for given label name. @@ -155,6 +193,9 @@ type UnimplementedStoreGatewayServer struct { func (*UnimplementedStoreGatewayServer) Series(req *storepb.SeriesRequest, srv StoreGateway_SeriesServer) error { return status.Errorf(codes.Unimplemented, "method Series not implemented") } +func (*UnimplementedStoreGatewayServer) Select(req *storepb.SelectRequest, srv StoreGateway_SelectServer) error { + return status.Errorf(codes.Unimplemented, "method Select not implemented") +} func (*UnimplementedStoreGatewayServer) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method LabelNames not implemented") } @@ -187,6 +228,27 @@ func (x *storeGatewaySeriesServer) Send(m *storepb.SeriesResponse) error { return x.ServerStream.SendMsg(m) } +func _StoreGateway_Select_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(storepb.SelectRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(StoreGatewayServer).Select(m, &storeGatewaySelectServer{stream}) +} + +type StoreGateway_SelectServer interface { + Send(*storepb.SelectResponse) error + grpc.ServerStream +} + +type storeGatewaySelectServer struct { + grpc.ServerStream +} + +func (x *storeGatewaySelectServer) Send(m *storepb.SelectResponse) error { + return x.ServerStream.SendMsg(m) +} + func _StoreGateway_LabelNames_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(storepb.LabelNamesRequest) if err := dec(in); err != nil { @@ -242,6 +304,115 @@ var _StoreGateway_serviceDesc = grpc.ServiceDesc{ Handler: _StoreGateway_Series_Handler, ServerStreams: true, }, + { + StreamName: "Select", + Handler: _StoreGateway_Select_Handler, + ServerStreams: true, + }, + }, + Metadata: "gateway.proto", +} + +// ChunksGatewayClient is the client API for ChunksGateway service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ChunksGatewayClient interface { + Chunks(ctx context.Context, opts ...grpc.CallOption) (ChunksGateway_ChunksClient, error) +} + +type chunksGatewayClient struct { + cc *grpc.ClientConn +} + +func NewChunksGatewayClient(cc *grpc.ClientConn) ChunksGatewayClient { + return &chunksGatewayClient{cc} +} + +func (c *chunksGatewayClient) Chunks(ctx context.Context, opts ...grpc.CallOption) (ChunksGateway_ChunksClient, error) { + stream, err := c.cc.NewStream(ctx, &_ChunksGateway_serviceDesc.Streams[0], "/gatewaypb.ChunksGateway/Chunks", opts...) + if err != nil { + return nil, err + } + x := &chunksGatewayChunksClient{stream} + return x, nil +} + +type ChunksGateway_ChunksClient interface { + Send(*storepb.ChunksRequest) error + Recv() (*storepb.ChunksResponse, error) + grpc.ClientStream +} + +type chunksGatewayChunksClient struct { + grpc.ClientStream +} + +func (x *chunksGatewayChunksClient) Send(m *storepb.ChunksRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *chunksGatewayChunksClient) Recv() (*storepb.ChunksResponse, error) { + m := new(storepb.ChunksResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// ChunksGatewayServer is the server API for ChunksGateway service. +type ChunksGatewayServer interface { + Chunks(ChunksGateway_ChunksServer) error +} + +// UnimplementedChunksGatewayServer can be embedded to have forward compatible implementations. +type UnimplementedChunksGatewayServer struct { +} + +func (*UnimplementedChunksGatewayServer) Chunks(srv ChunksGateway_ChunksServer) error { + return status.Errorf(codes.Unimplemented, "method Chunks not implemented") +} + +func RegisterChunksGatewayServer(s *grpc.Server, srv ChunksGatewayServer) { + s.RegisterService(&_ChunksGateway_serviceDesc, srv) +} + +func _ChunksGateway_Chunks_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ChunksGatewayServer).Chunks(&chunksGatewayChunksServer{stream}) +} + +type ChunksGateway_ChunksServer interface { + Send(*storepb.ChunksResponse) error + Recv() (*storepb.ChunksRequest, error) + grpc.ServerStream +} + +type chunksGatewayChunksServer struct { + grpc.ServerStream +} + +func (x *chunksGatewayChunksServer) Send(m *storepb.ChunksResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *chunksGatewayChunksServer) Recv() (*storepb.ChunksRequest, error) { + m := new(storepb.ChunksRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _ChunksGateway_serviceDesc = grpc.ServiceDesc{ + ServiceName: "gatewaypb.ChunksGateway", + HandlerType: (*ChunksGatewayServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Chunks", + Handler: _ChunksGateway_Chunks_Handler, + ServerStreams: true, + ClientStreams: true, + }, }, Metadata: "gateway.proto", } diff --git a/pkg/storegateway/storegatewaypb/gateway.proto b/pkg/storegateway/storegatewaypb/gateway.proto index 14e65859c2..978f0b2a7e 100644 --- a/pkg/storegateway/storegatewaypb/gateway.proto +++ b/pkg/storegateway/storegatewaypb/gateway.proto @@ -1,7 +1,8 @@ syntax = "proto3"; package gatewaypb; -import "github.com/thanos-io/thanos/pkg/store/storepb/rpc.proto"; +import "github.com/cortexproject/cortex/pkg/storegateway/storepb/rpc.proto"; +import "github.com/cortexproject/cortex/pkg/storegateway/typespb/types.proto"; option go_package = "storegatewaypb"; @@ -13,11 +14,21 @@ service StoreGateway { // be sent for previous one. // // Series are sorted. - rpc Series(thanos.SeriesRequest) returns (stream thanos.SeriesResponse); + rpc Series(storepb.SeriesRequest) returns (stream storepb.SeriesResponse); + + rpc Select(storepb.SelectRequest) returns (stream storepb.SelectResponse); + + // LabelNames returns all label names that is available. - rpc LabelNames(thanos.LabelNamesRequest) returns (thanos.LabelNamesResponse); + rpc LabelNames(storepb.LabelNamesRequest) returns (storepb.LabelNamesResponse); // LabelValues returns all label values for given label name. - rpc LabelValues(thanos.LabelValuesRequest) returns (thanos.LabelValuesResponse); + rpc LabelValues(storepb.LabelValuesRequest) returns (storepb.LabelValuesResponse); + } + + +service ChunksGateway { + rpc Chunks(stream storepb.ChunksRequest) returns (stream storepb.ChunksResponse); +} \ No newline at end of file diff --git a/pkg/storegateway/storepb/custom.go b/pkg/storegateway/storepb/custom.go new file mode 100644 index 0000000000..0abb8ef48e --- /dev/null +++ b/pkg/storegateway/storepb/custom.go @@ -0,0 +1,374 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storepb + +import ( + "strings" + + typespb "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/gogo/protobuf/types" + "github.com/prometheus/prometheus/model/labels" + "google.golang.org/grpc/codes" + + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +func NewWarnSeriesResponse(err error) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_Warning{ + Warning: err.Error(), + }, + } +} + +func NewSeriesResponse(series *typespb.Series) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_Series{ + Series: series, + }, + } +} + +func NewHintsSeriesResponse(hints *types.Any) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_Hints{ + Hints: hints, + }, + } +} + +func NewWarnSelectResponse(err error) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_Warning{ + Warning: err.Error(), + }, + } +} + +func NewSelectResponse(series *typespb.SelectedSeries) *SelectResponse { + return &SelectResponse{ + Result: &SelectResponse_Series{ + Series: series, + }, + } +} + +func NewHintsSelectResponse(hints *types.Any) *SelectResponse { + return &SelectResponse{ + Result: &SelectResponse_Hints{ + Hints: hints, + }, + } +} + +func GRPCCodeFromWarn(warn string) codes.Code { + if strings.Contains(warn, "rpc error: code = ResourceExhausted") { + return codes.ResourceExhausted + } + if strings.Contains(warn, "rpc error: code = Code(422)") { + return 422 + } + return codes.Unknown +} + +type emptySeriesSet struct{} + +func (emptySeriesSet) Next() bool { return false } +func (emptySeriesSet) At() (labels.Labels, []typespb.AggrChunk) { return nil, nil } +func (emptySeriesSet) Err() error { return nil } + +// EmptySeriesSet returns a new series set that contains no series. +func EmptySeriesSet() SeriesSet { + return emptySeriesSet{} +} + +// MergeSeriesSets takes all series sets and returns as a union single series set. +// It assumes series are sorted by labels within single SeriesSet, similar to remote read guarantees. +// However, they can be partial: in such case, if the single SeriesSet returns the same series within many iterations, +// MergeSeriesSets will merge those into one. +// +// It also assumes in a "best effort" way that chunks are sorted by min time. It's done as an optimization only, so if input +// series' chunks are NOT sorted, the only consequence is that the duplicates might be not correctly removed. This is double checked +// which on just-before PromQL level as well, so the only consequence is increased network bandwidth. +// If all chunks were sorted, MergeSeriesSet ALSO returns sorted chunks by min time. +// +// Chunks within the same series can also overlap (within all SeriesSet +// as well as single SeriesSet alone). If the chunk ranges overlap, the *exact* chunk duplicates will be removed +// (except one), and any other overlaps will be appended into on chunks slice. +func MergeSeriesSets(all ...SeriesSet) SeriesSet { + switch len(all) { + case 0: + return emptySeriesSet{} + case 1: + return newUniqueSeriesSet(all[0]) + } + h := len(all) / 2 + + return newMergedSeriesSet( + MergeSeriesSets(all[:h]...), + MergeSeriesSets(all[h:]...), + ) +} + +// SeriesSet is a set of series and their corresponding chunks. +// The set is sorted by the label sets. Chunks may be overlapping or expected of order. +type SeriesSet interface { + Next() bool + At() (labels.Labels, []typespb.AggrChunk) + Err() error +} + +// mergedSeriesSet takes two series sets as a single series set. +type mergedSeriesSet struct { + a, b SeriesSet + + lset labels.Labels + chunks []typespb.AggrChunk + adone, bdone bool +} + +func newMergedSeriesSet(a, b SeriesSet) *mergedSeriesSet { + s := &mergedSeriesSet{a: a, b: b} + // Initialize first elements of both sets as Next() needs + // one element look-ahead. + s.adone = !s.a.Next() + s.bdone = !s.b.Next() + + return s +} + +func (s *mergedSeriesSet) At() (labels.Labels, []typespb.AggrChunk) { + return s.lset, s.chunks +} + +func (s *mergedSeriesSet) Err() error { + if s.a.Err() != nil { + return s.a.Err() + } + return s.b.Err() +} + +func (s *mergedSeriesSet) compare() int { + if s.adone { + return 1 + } + if s.bdone { + return -1 + } + lsetA, _ := s.a.At() + lsetB, _ := s.b.At() + return labels.Compare(lsetA, lsetB) +} + +func (s *mergedSeriesSet) Next() bool { + if s.adone && s.bdone || s.Err() != nil { + return false + } + + d := s.compare() + if d > 0 { + s.lset, s.chunks = s.b.At() + s.bdone = !s.b.Next() + return true + } + if d < 0 { + s.lset, s.chunks = s.a.At() + s.adone = !s.a.Next() + return true + } + + // Both a and b contains the same series. Go through all chunks, remove duplicates and concatenate chunks from both + // series sets. We best effortly assume chunks are sorted by min time. If not, we will not detect all deduplicate which will + // be account on select layer anyway. We do it still for early optimization. + lset, chksA := s.a.At() + _, chksB := s.b.At() + s.lset = lset + + // Slice reuse is not generally safe with nested merge iterators. + // We err on the safe side an create a new slice. + s.chunks = make([]typespb.AggrChunk, 0, len(chksA)+len(chksB)) + + b := 0 +Outer: + for a := range chksA { + for { + if b >= len(chksB) { + // No more b chunks. + s.chunks = append(s.chunks, chksA[a:]...) + break Outer + } + + cmp := chksA[a].Compare(chksB[b]) + if cmp > 0 { + s.chunks = append(s.chunks, chksA[a]) + break + } + if cmp < 0 { + s.chunks = append(s.chunks, chksB[b]) + b++ + continue + } + + // Exact duplicated chunks, discard one from b. + b++ + } + } + + if b < len(chksB) { + s.chunks = append(s.chunks, chksB[b:]...) + } + + s.adone = !s.a.Next() + s.bdone = !s.b.Next() + return true +} + +// uniqueSeriesSet takes one series set and ensures each iteration contains single, full series. +type uniqueSeriesSet struct { + SeriesSet + done bool + + peek *typespb.Series + + lset labels.Labels + chunks []typespb.AggrChunk +} + +func newUniqueSeriesSet(wrapped SeriesSet) *uniqueSeriesSet { + return &uniqueSeriesSet{SeriesSet: wrapped} +} + +func (s *uniqueSeriesSet) At() (labels.Labels, []typespb.AggrChunk) { + return s.lset, s.chunks +} + +func (s *uniqueSeriesSet) Next() bool { + if s.Err() != nil { + return false + } + + for !s.done { + if s.done = !s.SeriesSet.Next(); s.done { + break + } + lset, chks := s.SeriesSet.At() + if s.peek == nil { + s.peek = &typespb.Series{Labels: labelpb.ZLabelsFromPromLabels(lset), Chunks: chks} + continue + } + + if labels.Compare(lset, s.peek.PromLabels()) != 0 { + s.lset, s.chunks = s.peek.PromLabels(), s.peek.Chunks + s.peek = &typespb.Series{Labels: labelpb.ZLabelsFromPromLabels(lset), Chunks: chks} + return true + } + + // We assume non-overlapping, sorted chunks. This is best effort only, if it's otherwise it + // will just be duplicated, but well handled by StoreAPI consumers. + s.peek.Chunks = append(s.peek.Chunks, chks...) + } + + if s.peek == nil { + return false + } + + s.lset, s.chunks = s.peek.PromLabels(), s.peek.Chunks + s.peek = nil + return true +} + +// Deprecated. +// TODO(bwplotka): Remove this once Cortex dep will stop using it. +type Label = labelpb.ZLabel + +// Deprecated. +// TODO(bwplotka): Remove this in next PR. Done to reduce diff only. +type LabelSet = labelpb.ZLabelSet + +// Deprecated. +// TODO(bwplotka): Remove this once Cortex dep will stop using it. +func CompareLabels(a, b []Label) int { + return labels.Compare(labelpb.ZLabelsToPromLabels(a), labelpb.ZLabelsToPromLabels(b)) +} + +// Deprecated. +// TODO(bwplotka): Remove this once Cortex dep will stop using it. +func LabelsToPromLabelsUnsafe(lset []Label) labels.Labels { + return labelpb.ZLabelsToPromLabels(lset) +} + +type SeriesStatsCounter struct { + lastSeriesHash uint64 + + Series int + Chunks int + Samples int +} + +func (c *SeriesStatsCounter) CountSeries(seriesLabels []labelpb.ZLabel) { + seriesHash := labelpb.HashWithPrefix("", seriesLabels) + if c.lastSeriesHash != 0 || seriesHash != c.lastSeriesHash { + c.lastSeriesHash = seriesHash + c.Series++ + } +} + +func (c *SeriesStatsCounter) Count(series *typespb.Series) { + c.CountSeries(series.Labels) + for _, chk := range series.Chunks { + if chk.Raw != nil { + c.Chunks++ + c.Samples += chk.Raw.XORNumSamples() + } + + if chk.Count != nil { + c.Chunks++ + c.Samples += chk.Count.XORNumSamples() + } + + if chk.Counter != nil { + c.Chunks++ + c.Samples += chk.Counter.XORNumSamples() + } + + if chk.Max != nil { + c.Chunks++ + c.Samples += chk.Max.XORNumSamples() + } + + if chk.Min != nil { + c.Chunks++ + c.Samples += chk.Min.XORNumSamples() + } + + if chk.Sum != nil { + c.Chunks++ + c.Samples += chk.Sum.XORNumSamples() + } + } +} + +func (m *SeriesRequest) ToPromQL() string { + return m.QueryHints.toPromQL(m.Matchers) +} + +// IsSafeToExecute returns true if the function or aggregation from the query hint +// can be safely executed by the underlying Prometheus instance without affecting the +// result of the query. +func (m *QueryHints) IsSafeToExecute() bool { + distributiveOperations := []string{ + "max", + "max_over_time", + "min", + "min_over_time", + "group", + } + for _, op := range distributiveOperations { + if m.Func.Name == op { + return true + } + } + + return false +} diff --git a/pkg/storegateway/storepb/query_hints.go b/pkg/storegateway/storepb/query_hints.go new file mode 100644 index 0000000000..aea27a7c56 --- /dev/null +++ b/pkg/storegateway/storepb/query_hints.go @@ -0,0 +1,50 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storepb + +import ( + "fmt" + "strings" + + typespb "github.com/cortexproject/cortex/pkg/storegateway/typespb" +) + +func (m *QueryHints) toPromQL(labelMatchers []typespb.LabelMatcher) string { + grouping := m.Grouping.toPromQL() + matchers := typespb.MatchersToString(labelMatchers...) + queryRange := m.Range.toPromQL() + + query := fmt.Sprintf("%s %s (%s%s)", m.Func.Name, grouping, matchers, queryRange) + // Remove double spaces if some expressions are missing. + return strings.Join(strings.Fields(query), " ") +} + +func (m *Grouping) toPromQL() string { + if m == nil { + return "" + } + + if len(m.Labels) == 0 { + return "" + } + var op string + if m.By { + op = "by" + } else { + op = "without" + } + + return fmt.Sprintf("%s (%s)", op, strings.Join(m.Labels, ",")) +} + +func (m *Range) toPromQL() string { + if m == nil { + return "" + } + + if m.Millis == 0 { + return "" + } + return fmt.Sprintf("[%dms]", m.Millis) +} diff --git a/pkg/storegateway/storepb/rpc.pb.go b/pkg/storegateway/storepb/rpc.pb.go new file mode 100644 index 0000000000..c5a7c08bff --- /dev/null +++ b/pkg/storegateway/storepb/rpc.pb.go @@ -0,0 +1,7085 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: rpc.proto + +package storepb + +import ( + context "context" + fmt "fmt" + typespb "github.com/cortexproject/cortex/pkg/storegateway/typespb" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + types "github.com/gogo/protobuf/types" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strconv "strconv" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type Aggr int32 + +const ( + RAW Aggr = 0 + COUNT Aggr = 1 + SUM Aggr = 2 + MIN Aggr = 3 + MAX Aggr = 4 + COUNTER Aggr = 5 +) + +var Aggr_name = map[int32]string{ + 0: "RAW", + 1: "COUNT", + 2: "SUM", + 3: "MIN", + 4: "MAX", + 5: "COUNTER", +} + +var Aggr_value = map[string]int32{ + "RAW": 0, + "COUNT": 1, + "SUM": 2, + "MIN": 3, + "MAX": 4, + "COUNTER": 5, +} + +func (Aggr) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{0} +} + +type SeriesRequest struct { + MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` + MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` + Matchers []typespb.LabelMatcher `protobuf:"bytes,3,rep,name=matchers,proto3" json:"matchers"` + MaxResolutionWindow int64 `protobuf:"varint,4,opt,name=max_resolution_window,json=maxResolutionWindow,proto3" json:"max_resolution_window,omitempty"` + Aggregates []Aggr `protobuf:"varint,5,rep,packed,name=aggregates,proto3,enum=storepb.Aggr" json:"aggregates,omitempty"` + // Deprecated. Use partial_response_strategy instead. + PartialResponseDisabled bool `protobuf:"varint,6,opt,name=partial_response_disabled,json=partialResponseDisabled,proto3" json:"partial_response_disabled,omitempty"` + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + PartialResponseStrategy typespb.PartialResponseStrategy `protobuf:"varint,7,opt,name=partial_response_strategy,json=partialResponseStrategy,proto3,enum=typespb.PartialResponseStrategy" json:"partial_response_strategy,omitempty"` + // skip_chunks controls whether sending chunks or not in series responses. + SkipChunks bool `protobuf:"varint,8,opt,name=skip_chunks,json=skipChunks,proto3" json:"skip_chunks,omitempty"` + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + Hints *types.Any `protobuf:"bytes,9,opt,name=hints,proto3" json:"hints,omitempty"` + // Query step size in milliseconds. + // Deprecated: Use query_hints instead. + Step int64 `protobuf:"varint,10,opt,name=step,proto3" json:"step,omitempty"` + // Range vector selector range in milliseconds. + // Deprecated: Use query_hints instead. + Range int64 `protobuf:"varint,11,opt,name=range,proto3" json:"range,omitempty"` + // query_hints are the hints coming from the PromQL engine when + // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. + QueryHints *QueryHints `protobuf:"bytes,12,opt,name=query_hints,json=queryHints,proto3" json:"query_hints,omitempty"` + // shard_info is used by the querier to request a specific + // shard of blocks instead of entire blocks. + ShardInfo *ShardInfo `protobuf:"bytes,13,opt,name=shard_info,json=shardInfo,proto3" json:"shard_info,omitempty"` + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + WithoutReplicaLabels []string `protobuf:"bytes,14,rep,name=without_replica_labels,json=withoutReplicaLabels,proto3" json:"without_replica_labels,omitempty"` +} + +func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } +func (*SeriesRequest) ProtoMessage() {} +func (*SeriesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{0} +} +func (m *SeriesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesRequest.Merge(m, src) +} +func (m *SeriesRequest) XXX_Size() int { + return m.Size() +} +func (m *SeriesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesRequest proto.InternalMessageInfo + +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. +type QueryHints struct { + // Query step size in milliseconds. + StepMillis int64 `protobuf:"varint,1,opt,name=step_millis,json=stepMillis,proto3" json:"step_millis,omitempty"` + // The surrounding function or aggregation. + Func *Func `protobuf:"bytes,2,opt,name=func,proto3" json:"func,omitempty"` + // The grouping expression + Grouping *Grouping `protobuf:"bytes,4,opt,name=grouping,proto3" json:"grouping,omitempty"` + // Range vector selector. + Range *Range `protobuf:"bytes,5,opt,name=range,proto3" json:"range,omitempty"` +} + +func (m *QueryHints) Reset() { *m = QueryHints{} } +func (*QueryHints) ProtoMessage() {} +func (*QueryHints) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{1} +} +func (m *QueryHints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryHints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_QueryHints.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *QueryHints) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryHints.Merge(m, src) +} +func (m *QueryHints) XXX_Size() int { + return m.Size() +} +func (m *QueryHints) XXX_DiscardUnknown() { + xxx_messageInfo_QueryHints.DiscardUnknown(m) +} + +var xxx_messageInfo_QueryHints proto.InternalMessageInfo + +// ShardInfo are the parameters used to shard series in Stores. +type ShardInfo struct { + // The index of the current shard. + ShardIndex int64 `protobuf:"varint,1,opt,name=shard_index,json=shardIndex,proto3" json:"shard_index,omitempty"` + // The total number of shards. + TotalShards int64 `protobuf:"varint,2,opt,name=total_shards,json=totalShards,proto3" json:"total_shards,omitempty"` + // Group by or without labels. + By bool `protobuf:"varint,3,opt,name=by,proto3" json:"by,omitempty"` + // Labels on which to partition series. + Labels []string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty"` +} + +func (m *ShardInfo) Reset() { *m = ShardInfo{} } +func (*ShardInfo) ProtoMessage() {} +func (*ShardInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{2} +} +func (m *ShardInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ShardInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ShardInfo.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ShardInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShardInfo.Merge(m, src) +} +func (m *ShardInfo) XXX_Size() int { + return m.Size() +} +func (m *ShardInfo) XXX_DiscardUnknown() { + xxx_messageInfo_ShardInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_ShardInfo proto.InternalMessageInfo + +type Func struct { + // The function or aggregation name + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` +} + +func (m *Func) Reset() { *m = Func{} } +func (*Func) ProtoMessage() {} +func (*Func) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{3} +} +func (m *Func) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Func) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Func.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Func) XXX_Merge(src proto.Message) { + xxx_messageInfo_Func.Merge(m, src) +} +func (m *Func) XXX_Size() int { + return m.Size() +} +func (m *Func) XXX_DiscardUnknown() { + xxx_messageInfo_Func.DiscardUnknown(m) +} + +var xxx_messageInfo_Func proto.InternalMessageInfo + +type Grouping struct { + // Indicate whether it is without or by. + By bool `protobuf:"varint,1,opt,name=by,proto3" json:"by,omitempty"` + // List of label names used in the grouping. + Labels []string `protobuf:"bytes,3,rep,name=labels,proto3" json:"labels,omitempty"` +} + +func (m *Grouping) Reset() { *m = Grouping{} } +func (*Grouping) ProtoMessage() {} +func (*Grouping) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{4} +} +func (m *Grouping) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Grouping) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Grouping.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Grouping) XXX_Merge(src proto.Message) { + xxx_messageInfo_Grouping.Merge(m, src) +} +func (m *Grouping) XXX_Size() int { + return m.Size() +} +func (m *Grouping) XXX_DiscardUnknown() { + xxx_messageInfo_Grouping.DiscardUnknown(m) +} + +var xxx_messageInfo_Grouping proto.InternalMessageInfo + +type Range struct { + Millis int64 `protobuf:"varint,1,opt,name=millis,proto3" json:"millis,omitempty"` +} + +func (m *Range) Reset() { *m = Range{} } +func (*Range) ProtoMessage() {} +func (*Range) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{5} +} +func (m *Range) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Range) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Range.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Range) XXX_Merge(src proto.Message) { + xxx_messageInfo_Range.Merge(m, src) +} +func (m *Range) XXX_Size() int { + return m.Size() +} +func (m *Range) XXX_DiscardUnknown() { + xxx_messageInfo_Range.DiscardUnknown(m) +} + +var xxx_messageInfo_Range proto.InternalMessageInfo + +type SeriesResponse struct { + // Types that are valid to be assigned to Result: + // *SeriesResponse_Series + // *SeriesResponse_Warning + // *SeriesResponse_Hints + Result isSeriesResponse_Result `protobuf_oneof:"result"` +} + +func (m *SeriesResponse) Reset() { *m = SeriesResponse{} } +func (*SeriesResponse) ProtoMessage() {} +func (*SeriesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{6} +} +func (m *SeriesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesResponse.Merge(m, src) +} +func (m *SeriesResponse) XXX_Size() int { + return m.Size() +} +func (m *SeriesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesResponse proto.InternalMessageInfo + +type isSeriesResponse_Result interface { + isSeriesResponse_Result() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type SeriesResponse_Series struct { + Series *typespb.Series `protobuf:"bytes,1,opt,name=series,proto3,oneof"` +} +type SeriesResponse_Warning struct { + Warning string `protobuf:"bytes,2,opt,name=warning,proto3,oneof"` +} +type SeriesResponse_Hints struct { + Hints *types.Any `protobuf:"bytes,3,opt,name=hints,proto3,oneof"` +} + +func (*SeriesResponse_Series) isSeriesResponse_Result() {} +func (*SeriesResponse_Warning) isSeriesResponse_Result() {} +func (*SeriesResponse_Hints) isSeriesResponse_Result() {} + +func (m *SeriesResponse) GetResult() isSeriesResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *SeriesResponse) GetSeries() *typespb.Series { + if x, ok := m.GetResult().(*SeriesResponse_Series); ok { + return x.Series + } + return nil +} + +func (m *SeriesResponse) GetWarning() string { + if x, ok := m.GetResult().(*SeriesResponse_Warning); ok { + return x.Warning + } + return "" +} + +func (m *SeriesResponse) GetHints() *types.Any { + if x, ok := m.GetResult().(*SeriesResponse_Hints); ok { + return x.Hints + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SeriesResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SeriesResponse_Series)(nil), + (*SeriesResponse_Warning)(nil), + (*SeriesResponse_Hints)(nil), + } +} + +type LabelNamesRequest struct { + PartialResponseDisabled bool `protobuf:"varint,1,opt,name=partial_response_disabled,json=partialResponseDisabled,proto3" json:"partial_response_disabled,omitempty"` + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + PartialResponseStrategy typespb.PartialResponseStrategy `protobuf:"varint,2,opt,name=partial_response_strategy,json=partialResponseStrategy,proto3,enum=typespb.PartialResponseStrategy" json:"partial_response_strategy,omitempty"` + Start int64 `protobuf:"varint,3,opt,name=start,proto3" json:"start,omitempty"` + End int64 `protobuf:"varint,4,opt,name=end,proto3" json:"end,omitempty"` + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + Hints *types.Any `protobuf:"bytes,5,opt,name=hints,proto3" json:"hints,omitempty"` + Matchers []typespb.LabelMatcher `protobuf:"bytes,6,rep,name=matchers,proto3" json:"matchers"` +} + +func (m *LabelNamesRequest) Reset() { *m = LabelNamesRequest{} } +func (*LabelNamesRequest) ProtoMessage() {} +func (*LabelNamesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{7} +} +func (m *LabelNamesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelNamesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelNamesRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelNamesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelNamesRequest.Merge(m, src) +} +func (m *LabelNamesRequest) XXX_Size() int { + return m.Size() +} +func (m *LabelNamesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LabelNamesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelNamesRequest proto.InternalMessageInfo + +type LabelNamesResponse struct { + Names []string `protobuf:"bytes,1,rep,name=names,proto3" json:"names,omitempty"` + Warnings []string `protobuf:"bytes,2,rep,name=warnings,proto3" json:"warnings,omitempty"` + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. + Hints *types.Any `protobuf:"bytes,3,opt,name=hints,proto3" json:"hints,omitempty"` +} + +func (m *LabelNamesResponse) Reset() { *m = LabelNamesResponse{} } +func (*LabelNamesResponse) ProtoMessage() {} +func (*LabelNamesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{8} +} +func (m *LabelNamesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelNamesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelNamesResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelNamesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelNamesResponse.Merge(m, src) +} +func (m *LabelNamesResponse) XXX_Size() int { + return m.Size() +} +func (m *LabelNamesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LabelNamesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelNamesResponse proto.InternalMessageInfo + +type LabelValuesRequest struct { + Label string `protobuf:"bytes,1,opt,name=label,proto3" json:"label,omitempty"` + PartialResponseDisabled bool `protobuf:"varint,2,opt,name=partial_response_disabled,json=partialResponseDisabled,proto3" json:"partial_response_disabled,omitempty"` + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + PartialResponseStrategy typespb.PartialResponseStrategy `protobuf:"varint,3,opt,name=partial_response_strategy,json=partialResponseStrategy,proto3,enum=typespb.PartialResponseStrategy" json:"partial_response_strategy,omitempty"` + Start int64 `protobuf:"varint,4,opt,name=start,proto3" json:"start,omitempty"` + End int64 `protobuf:"varint,5,opt,name=end,proto3" json:"end,omitempty"` + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + Hints *types.Any `protobuf:"bytes,6,opt,name=hints,proto3" json:"hints,omitempty"` + Matchers []typespb.LabelMatcher `protobuf:"bytes,7,rep,name=matchers,proto3" json:"matchers"` +} + +func (m *LabelValuesRequest) Reset() { *m = LabelValuesRequest{} } +func (*LabelValuesRequest) ProtoMessage() {} +func (*LabelValuesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{9} +} +func (m *LabelValuesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelValuesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelValuesRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelValuesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelValuesRequest.Merge(m, src) +} +func (m *LabelValuesRequest) XXX_Size() int { + return m.Size() +} +func (m *LabelValuesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LabelValuesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelValuesRequest proto.InternalMessageInfo + +type LabelValuesResponse struct { + Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + Warnings []string `protobuf:"bytes,2,rep,name=warnings,proto3" json:"warnings,omitempty"` + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. + Hints *types.Any `protobuf:"bytes,3,opt,name=hints,proto3" json:"hints,omitempty"` +} + +func (m *LabelValuesResponse) Reset() { *m = LabelValuesResponse{} } +func (*LabelValuesResponse) ProtoMessage() {} +func (*LabelValuesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{10} +} +func (m *LabelValuesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelValuesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelValuesResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelValuesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelValuesResponse.Merge(m, src) +} +func (m *LabelValuesResponse) XXX_Size() int { + return m.Size() +} +func (m *LabelValuesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LabelValuesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelValuesResponse proto.InternalMessageInfo + +type SelectRequest struct { + MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` + MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` + Matchers []typespb.LabelMatcher `protobuf:"bytes,3,rep,name=matchers,proto3" json:"matchers"` + MaxResolutionWindow int64 `protobuf:"varint,4,opt,name=max_resolution_window,json=maxResolutionWindow,proto3" json:"max_resolution_window,omitempty"` + Aggregates []Aggr `protobuf:"varint,5,rep,packed,name=aggregates,proto3,enum=storepb.Aggr" json:"aggregates,omitempty"` + // Deprecated. Use partial_response_strategy instead. + PartialResponseDisabled bool `protobuf:"varint,6,opt,name=partial_response_disabled,json=partialResponseDisabled,proto3" json:"partial_response_disabled,omitempty"` + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + PartialResponseStrategy typespb.PartialResponseStrategy `protobuf:"varint,7,opt,name=partial_response_strategy,json=partialResponseStrategy,proto3,enum=typespb.PartialResponseStrategy" json:"partial_response_strategy,omitempty"` + // skip_chunks controls whether sending chunks or not in series responses. + SkipChunks bool `protobuf:"varint,8,opt,name=skip_chunks,json=skipChunks,proto3" json:"skip_chunks,omitempty"` + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + Hints *types.Any `protobuf:"bytes,9,opt,name=hints,proto3" json:"hints,omitempty"` + // Query step size in milliseconds. + // Deprecated: Use query_hints instead. + Step int64 `protobuf:"varint,10,opt,name=step,proto3" json:"step,omitempty"` + // Range vector selector range in milliseconds. + // Deprecated: Use query_hints instead. + Range int64 `protobuf:"varint,11,opt,name=range,proto3" json:"range,omitempty"` + // query_hints are the hints coming from the PromQL engine when + // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. + QueryHints *QueryHints `protobuf:"bytes,12,opt,name=query_hints,json=queryHints,proto3" json:"query_hints,omitempty"` + // shard_info is used by the querier to request a specific + // shard of blocks instead of entire blocks. + ShardInfo *ShardInfo `protobuf:"bytes,13,opt,name=shard_info,json=shardInfo,proto3" json:"shard_info,omitempty"` + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + WithoutReplicaLabels []string `protobuf:"bytes,14,rep,name=without_replica_labels,json=withoutReplicaLabels,proto3" json:"without_replica_labels,omitempty"` +} + +func (m *SelectRequest) Reset() { *m = SelectRequest{} } +func (*SelectRequest) ProtoMessage() {} +func (*SelectRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{11} +} +func (m *SelectRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SelectRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SelectRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SelectRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SelectRequest.Merge(m, src) +} +func (m *SelectRequest) XXX_Size() int { + return m.Size() +} +func (m *SelectRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SelectRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SelectRequest proto.InternalMessageInfo + +type SelectResponse struct { + // Types that are valid to be assigned to Result: + // *SelectResponse_Series + // *SelectResponse_Warning + // *SelectResponse_Hints + Result isSelectResponse_Result `protobuf_oneof:"result"` +} + +func (m *SelectResponse) Reset() { *m = SelectResponse{} } +func (*SelectResponse) ProtoMessage() {} +func (*SelectResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{12} +} +func (m *SelectResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SelectResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SelectResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SelectResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SelectResponse.Merge(m, src) +} +func (m *SelectResponse) XXX_Size() int { + return m.Size() +} +func (m *SelectResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SelectResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SelectResponse proto.InternalMessageInfo + +type isSelectResponse_Result interface { + isSelectResponse_Result() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type SelectResponse_Series struct { + Series *typespb.SelectedSeries `protobuf:"bytes,1,opt,name=series,proto3,oneof"` +} +type SelectResponse_Warning struct { + Warning string `protobuf:"bytes,2,opt,name=warning,proto3,oneof"` +} +type SelectResponse_Hints struct { + Hints *types.Any `protobuf:"bytes,3,opt,name=hints,proto3,oneof"` +} + +func (*SelectResponse_Series) isSelectResponse_Result() {} +func (*SelectResponse_Warning) isSelectResponse_Result() {} +func (*SelectResponse_Hints) isSelectResponse_Result() {} + +func (m *SelectResponse) GetResult() isSelectResponse_Result { + if m != nil { + return m.Result + } + return nil +} + +func (m *SelectResponse) GetSeries() *typespb.SelectedSeries { + if x, ok := m.GetResult().(*SelectResponse_Series); ok { + return x.Series + } + return nil +} + +func (m *SelectResponse) GetWarning() string { + if x, ok := m.GetResult().(*SelectResponse_Warning); ok { + return x.Warning + } + return "" +} + +func (m *SelectResponse) GetHints() *types.Any { + if x, ok := m.GetResult().(*SelectResponse_Hints); ok { + return x.Hints + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SelectResponse) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SelectResponse_Series)(nil), + (*SelectResponse_Warning)(nil), + (*SelectResponse_Hints)(nil), + } +} + +type ChunksRequest struct { + BlockId string `protobuf:"bytes,1,opt,name=blockId,proto3" json:"blockId,omitempty"` + Chunkref []uint64 `protobuf:"varint,2,rep,packed,name=chunkref,proto3" json:"chunkref,omitempty"` +} + +func (m *ChunksRequest) Reset() { *m = ChunksRequest{} } +func (*ChunksRequest) ProtoMessage() {} +func (*ChunksRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{13} +} +func (m *ChunksRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChunksRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ChunksRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ChunksRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChunksRequest.Merge(m, src) +} +func (m *ChunksRequest) XXX_Size() int { + return m.Size() +} +func (m *ChunksRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ChunksRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ChunksRequest proto.InternalMessageInfo + +type ChunksResponse struct { + Chunks []*typespb.AggrChunk `protobuf:"bytes,1,rep,name=chunks,proto3" json:"chunks,omitempty"` +} + +func (m *ChunksResponse) Reset() { *m = ChunksResponse{} } +func (*ChunksResponse) ProtoMessage() {} +func (*ChunksResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_77a6da22d6a3feb1, []int{14} +} +func (m *ChunksResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChunksResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ChunksResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ChunksResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChunksResponse.Merge(m, src) +} +func (m *ChunksResponse) XXX_Size() int { + return m.Size() +} +func (m *ChunksResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ChunksResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ChunksResponse proto.InternalMessageInfo + +func init() { + proto.RegisterEnum("storepb.Aggr", Aggr_name, Aggr_value) + proto.RegisterType((*SeriesRequest)(nil), "storepb.SeriesRequest") + proto.RegisterType((*QueryHints)(nil), "storepb.QueryHints") + proto.RegisterType((*ShardInfo)(nil), "storepb.ShardInfo") + proto.RegisterType((*Func)(nil), "storepb.Func") + proto.RegisterType((*Grouping)(nil), "storepb.Grouping") + proto.RegisterType((*Range)(nil), "storepb.Range") + proto.RegisterType((*SeriesResponse)(nil), "storepb.SeriesResponse") + proto.RegisterType((*LabelNamesRequest)(nil), "storepb.LabelNamesRequest") + proto.RegisterType((*LabelNamesResponse)(nil), "storepb.LabelNamesResponse") + proto.RegisterType((*LabelValuesRequest)(nil), "storepb.LabelValuesRequest") + proto.RegisterType((*LabelValuesResponse)(nil), "storepb.LabelValuesResponse") + proto.RegisterType((*SelectRequest)(nil), "storepb.SelectRequest") + proto.RegisterType((*SelectResponse)(nil), "storepb.SelectResponse") + proto.RegisterType((*ChunksRequest)(nil), "storepb.ChunksRequest") + proto.RegisterType((*ChunksResponse)(nil), "storepb.ChunksResponse") +} + +func init() { proto.RegisterFile("rpc.proto", fileDescriptor_77a6da22d6a3feb1) } + +var fileDescriptor_77a6da22d6a3feb1 = []byte{ + // 1217 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6e, 0xdb, 0x46, + 0x10, 0x26, 0x45, 0x51, 0x3f, 0xa3, 0x58, 0x55, 0xd6, 0x8a, 0x4d, 0xab, 0x05, 0xad, 0x08, 0x3d, + 0xa8, 0x46, 0x23, 0x39, 0x6a, 0x80, 0x02, 0xfd, 0x39, 0xd8, 0x89, 0x5b, 0x1b, 0xad, 0xdd, 0x96, + 0x4e, 0x9a, 0xa2, 0x28, 0x20, 0x50, 0xd2, 0x9a, 0x62, 0x4d, 0x91, 0x34, 0x77, 0x59, 0x49, 0xb7, + 0xbe, 0x41, 0x7f, 0x2e, 0x7d, 0x82, 0x02, 0x7d, 0x86, 0x3e, 0x81, 0x8f, 0xee, 0x2d, 0xa7, 0xa2, + 0x96, 0x2f, 0x3d, 0xe6, 0x11, 0x8a, 0x5d, 0x2e, 0x29, 0x29, 0x56, 0x13, 0x18, 0x0e, 0x8a, 0x1e, + 0x72, 0xd2, 0xce, 0x7c, 0xc3, 0xe1, 0xce, 0x7c, 0xdf, 0xec, 0x52, 0x90, 0x0f, 0xfc, 0x6e, 0xc3, + 0x0f, 0x3c, 0xea, 0xa1, 0x2c, 0xa1, 0x5e, 0x80, 0xfd, 0x4e, 0xa5, 0x6c, 0x79, 0x96, 0xc7, 0x7d, + 0x4d, 0xb6, 0x8a, 0xe0, 0xca, 0x03, 0xcb, 0xa6, 0xfd, 0xb0, 0xd3, 0xe8, 0x7a, 0x83, 0x66, 0xd7, + 0x0b, 0x28, 0x1e, 0xf9, 0x81, 0xf7, 0x2d, 0xee, 0x52, 0x61, 0x35, 0xfd, 0x63, 0xab, 0xc9, 0x53, + 0x58, 0x26, 0xc5, 0x43, 0x73, 0xdc, 0xa4, 0x63, 0x1f, 0x13, 0xbf, 0x13, 0xfd, 0x8a, 0x2c, 0x6b, + 0x96, 0xe7, 0x59, 0x0e, 0x6e, 0x72, 0xab, 0x13, 0x1e, 0x35, 0x4d, 0x77, 0x1c, 0x41, 0xb5, 0x9f, + 0x54, 0x58, 0x3a, 0xc4, 0x81, 0x8d, 0x89, 0x81, 0x4f, 0x42, 0x4c, 0x28, 0x5a, 0x83, 0xdc, 0xc0, + 0x76, 0xdb, 0xd4, 0x1e, 0x60, 0x4d, 0xae, 0xca, 0x75, 0xc5, 0xc8, 0x0e, 0x6c, 0xf7, 0xa1, 0x3d, + 0xc0, 0x1c, 0x32, 0x47, 0x11, 0x94, 0x12, 0x90, 0x39, 0xe2, 0xd0, 0xbb, 0x0c, 0xa2, 0xdd, 0x3e, + 0x0e, 0x88, 0xa6, 0x54, 0x95, 0x7a, 0xa1, 0x75, 0xab, 0x21, 0xb6, 0xd2, 0xf8, 0xd4, 0xec, 0x60, + 0x67, 0x3f, 0x42, 0xb7, 0xd3, 0xa7, 0x7f, 0xae, 0x4b, 0x46, 0x12, 0x8c, 0x5a, 0x70, 0x8b, 0xe5, + 0x0c, 0x30, 0xf1, 0x9c, 0x90, 0xda, 0x9e, 0xdb, 0x1e, 0xda, 0x6e, 0xcf, 0x1b, 0x6a, 0x69, 0xfe, + 0x82, 0xe5, 0x81, 0x39, 0x32, 0x12, 0xec, 0x31, 0x87, 0xd0, 0x1d, 0x00, 0xd3, 0xb2, 0xa2, 0x9a, + 0x89, 0xa6, 0x56, 0x95, 0x7a, 0xb1, 0xb5, 0xd4, 0x10, 0x9d, 0x6c, 0x6c, 0x59, 0x56, 0x60, 0xcc, + 0x04, 0xa0, 0xf7, 0x60, 0xcd, 0x37, 0x03, 0x6a, 0x9b, 0x0e, 0x7b, 0x8d, 0xef, 0xb9, 0x04, 0xb7, + 0x7b, 0x36, 0x31, 0x3b, 0x0e, 0xee, 0x69, 0x99, 0xaa, 0x5c, 0xcf, 0x19, 0xab, 0x22, 0xc0, 0x10, + 0xf8, 0x03, 0x01, 0xa3, 0x6f, 0x16, 0x3c, 0x4b, 0x68, 0x60, 0x52, 0x6c, 0x8d, 0xb5, 0x6c, 0x55, + 0xae, 0x17, 0x5b, 0xd5, 0xa4, 0xd0, 0xcf, 0xe7, 0x93, 0x1c, 0x8a, 0xb8, 0x4b, 0xd9, 0x63, 0x00, + 0xad, 0x43, 0x81, 0x1c, 0xdb, 0x7e, 0xbb, 0xdb, 0x0f, 0xdd, 0x63, 0xa2, 0xe5, 0xf8, 0x5e, 0x80, + 0xb9, 0xee, 0x73, 0x0f, 0xda, 0x00, 0xb5, 0x6f, 0xbb, 0x94, 0x68, 0xf9, 0xaa, 0x5c, 0x2f, 0xb4, + 0xca, 0x8d, 0x88, 0xc9, 0x46, 0xcc, 0x64, 0x63, 0xcb, 0x1d, 0x1b, 0x51, 0x08, 0x42, 0x90, 0x26, + 0x14, 0xfb, 0x1a, 0xf0, 0xc6, 0xf1, 0x35, 0x2a, 0x83, 0x1a, 0x98, 0xae, 0x85, 0xb5, 0x02, 0x77, + 0x46, 0x06, 0xba, 0x07, 0x85, 0x93, 0x10, 0x07, 0xe3, 0x76, 0x94, 0xfb, 0x06, 0xcf, 0xbd, 0x9c, + 0x34, 0xf0, 0x0b, 0x86, 0xed, 0x32, 0xc8, 0x80, 0x93, 0x64, 0x8d, 0xee, 0x02, 0x90, 0xbe, 0x19, + 0xf4, 0xda, 0xb6, 0x7b, 0xe4, 0x69, 0x4b, 0xfc, 0x21, 0x94, 0x3c, 0x74, 0xc8, 0xa0, 0x3d, 0xf7, + 0xc8, 0x33, 0xf2, 0x24, 0x5e, 0xa2, 0x7b, 0xb0, 0x32, 0xb4, 0x69, 0xdf, 0x0b, 0x69, 0x3b, 0xc0, + 0xbe, 0x63, 0x77, 0xcd, 0xb6, 0xc3, 0xc4, 0x40, 0xb4, 0x62, 0x55, 0xa9, 0xe7, 0x8d, 0xb2, 0x40, + 0x8d, 0x08, 0xe4, 0x42, 0x21, 0xb5, 0x5f, 0x65, 0x80, 0xe9, 0x1e, 0x78, 0x93, 0x28, 0xf6, 0xdb, + 0x03, 0xdb, 0x71, 0x6c, 0x22, 0x34, 0x09, 0xcc, 0xb5, 0xcf, 0x3d, 0xe8, 0x36, 0xa4, 0x8f, 0x42, + 0xb7, 0xcb, 0x25, 0x59, 0x98, 0x11, 0xc2, 0x47, 0xa1, 0xdb, 0x35, 0x38, 0x84, 0xee, 0x40, 0xce, + 0x0a, 0xbc, 0xd0, 0xb7, 0x5d, 0x8b, 0x0b, 0xab, 0xd0, 0xba, 0x99, 0x84, 0x7d, 0x2c, 0x00, 0x23, + 0x09, 0x41, 0x6f, 0xc6, 0x6d, 0x53, 0x79, 0x6c, 0x31, 0x89, 0x35, 0x98, 0x57, 0xb4, 0xb1, 0x36, + 0x84, 0x7c, 0x52, 0x35, 0xdf, 0xa5, 0xe8, 0x4e, 0x0f, 0x8f, 0x92, 0x5d, 0x46, 0x78, 0x0f, 0x8f, + 0xd0, 0x6d, 0xb8, 0x41, 0x3d, 0x6a, 0x3a, 0x6d, 0xee, 0x23, 0x62, 0x80, 0x0a, 0xdc, 0xc7, 0xd3, + 0x10, 0x54, 0x84, 0x54, 0x67, 0xac, 0x29, 0x5c, 0x05, 0xa9, 0xce, 0x18, 0xad, 0x40, 0x46, 0xb4, + 0x2b, 0xcd, 0xdb, 0x25, 0xac, 0x5a, 0x05, 0xd2, 0xac, 0x36, 0xc6, 0xb8, 0x6b, 0x8a, 0x31, 0xcd, + 0x1b, 0x7c, 0x5d, 0x6b, 0x41, 0x2e, 0x2e, 0x48, 0xe4, 0x93, 0x17, 0xe4, 0x53, 0xe6, 0xf2, 0xad, + 0x83, 0xca, 0x0b, 0x63, 0x01, 0x73, 0x5d, 0x16, 0x56, 0xed, 0x07, 0x19, 0x8a, 0xf1, 0x29, 0x11, + 0x49, 0x18, 0xbd, 0x05, 0x19, 0xc2, 0x3d, 0x3c, 0xb4, 0xd0, 0x7a, 0x2d, 0x99, 0x82, 0x28, 0x70, + 0x57, 0x32, 0x44, 0x00, 0xaa, 0x40, 0x76, 0x68, 0x06, 0x2e, 0xeb, 0x3d, 0x2b, 0x3a, 0xbf, 0x2b, + 0x19, 0xb1, 0x03, 0xbd, 0x1d, 0x0b, 0x5c, 0xf9, 0x77, 0x81, 0xef, 0x4a, 0x42, 0xe2, 0xdb, 0x39, + 0xc8, 0x04, 0x98, 0x84, 0x0e, 0xad, 0xfd, 0x9e, 0x82, 0x9b, 0x5c, 0x2e, 0x07, 0xe6, 0x60, 0x7a, + 0x76, 0x3d, 0x77, 0xd2, 0xe5, 0x6b, 0x4c, 0x7a, 0xea, 0xba, 0x93, 0x5e, 0x06, 0x95, 0x50, 0x33, + 0xa0, 0xbc, 0x4e, 0xc5, 0x88, 0x0c, 0x54, 0x02, 0x05, 0xbb, 0x3d, 0x71, 0xd4, 0xb1, 0xe5, 0x74, + 0xe0, 0xd5, 0x17, 0x0f, 0xfc, 0xec, 0x99, 0x9b, 0xb9, 0xc2, 0x99, 0x5b, 0x0b, 0x00, 0xcd, 0xf6, + 0x4e, 0x30, 0x5a, 0x06, 0x95, 0x29, 0x88, 0x11, 0xca, 0xc4, 0x11, 0x19, 0xa8, 0x02, 0x39, 0xc1, + 0x15, 0x93, 0x2c, 0x03, 0x12, 0x7b, 0xba, 0x59, 0xe5, 0x85, 0x9b, 0xad, 0xfd, 0x91, 0x12, 0x2f, + 0xfd, 0xd2, 0x74, 0xc2, 0x29, 0x63, 0x65, 0x50, 0xb9, 0x08, 0x85, 0x86, 0x23, 0xe3, 0xf9, 0x3c, + 0xa6, 0xae, 0xc1, 0xa3, 0xf2, 0xd2, 0x78, 0x4c, 0x2f, 0xe0, 0x51, 0x5d, 0xc0, 0x63, 0xe6, 0x6a, + 0x3c, 0x66, 0xaf, 0xc2, 0x63, 0x08, 0xcb, 0x73, 0x2d, 0x15, 0x44, 0xae, 0x40, 0xe6, 0x3b, 0xee, + 0x11, 0x4c, 0x0a, 0xeb, 0xa5, 0x51, 0x19, 0x7d, 0x33, 0x38, 0xb8, 0x4b, 0x5f, 0x7d, 0x33, 0xbc, + 0xfa, 0x66, 0xf8, 0x7f, 0x7c, 0x33, 0xfc, 0xc2, 0x6f, 0xa8, 0x48, 0x93, 0x62, 0x0c, 0xee, 0x3e, + 0x73, 0x43, 0xad, 0xce, 0xdc, 0x50, 0x2c, 0x10, 0xf7, 0xfe, 0x93, 0x9b, 0x6a, 0x07, 0x96, 0x22, + 0x62, 0xe2, 0x61, 0xd1, 0x20, 0xdb, 0x71, 0xbc, 0xee, 0xf1, 0x5e, 0x4f, 0x1c, 0x7a, 0xb1, 0xc9, + 0x06, 0x94, 0xb3, 0x1a, 0xe0, 0x23, 0x3e, 0xa0, 0x69, 0x23, 0xb1, 0x6b, 0x1f, 0x40, 0x31, 0x4e, + 0x23, 0xea, 0xdb, 0x80, 0x8c, 0xd0, 0x80, 0xcc, 0x87, 0x07, 0x25, 0xf5, 0x31, 0x35, 0xf3, 0x60, + 0x43, 0x44, 0x6c, 0x6c, 0x43, 0x9a, 0x39, 0x51, 0x16, 0x14, 0x63, 0xeb, 0x71, 0x49, 0x42, 0x79, + 0x50, 0xef, 0x7f, 0xf6, 0xe8, 0xe0, 0x61, 0x49, 0x66, 0xbe, 0xc3, 0x47, 0xfb, 0xa5, 0x14, 0x5b, + 0xec, 0xef, 0x1d, 0x94, 0x14, 0xbe, 0xd8, 0xfa, 0xaa, 0x94, 0x46, 0x05, 0xc8, 0xf2, 0xa8, 0x1d, + 0xa3, 0xa4, 0xb6, 0x7e, 0x4e, 0x81, 0x7a, 0xc8, 0x98, 0x43, 0xef, 0x43, 0x26, 0x6a, 0x1d, 0x5a, + 0x99, 0x72, 0x39, 0xfb, 0x27, 0xa2, 0xb2, 0x7a, 0xc9, 0x1f, 0x6d, 0x7a, 0x53, 0x46, 0x3b, 0x00, + 0xd3, 0xcb, 0x07, 0x55, 0x92, 0xc0, 0x4b, 0xb7, 0x79, 0xe5, 0xf5, 0x85, 0x98, 0xa8, 0x7e, 0x17, + 0x0a, 0x33, 0x67, 0x1f, 0x7a, 0x26, 0x76, 0xee, 0x92, 0xa9, 0xbc, 0xb1, 0x18, 0x14, 0x99, 0x78, + 0x35, 0x4c, 0x10, 0x73, 0xd5, 0xcc, 0x1c, 0x6f, 0x73, 0xd5, 0xcc, 0x4a, 0x6c, 0x53, 0x6e, 0x7d, + 0x02, 0xc0, 0x3b, 0x1d, 0x35, 0xe6, 0x43, 0xc8, 0x88, 0x21, 0x9c, 0xa6, 0x9a, 0x23, 0x7f, 0x26, + 0xd5, 0x3c, 0x9b, 0x75, 0x79, 0x53, 0xde, 0xde, 0x3a, 0x3d, 0xd7, 0xa5, 0xb3, 0x73, 0x5d, 0x7a, + 0x72, 0xae, 0x4b, 0x4f, 0xcf, 0x75, 0xf9, 0xfb, 0x89, 0x2e, 0xff, 0x36, 0xd1, 0xe5, 0xd3, 0x89, + 0x2e, 0x9f, 0x4d, 0x74, 0xf9, 0xaf, 0x89, 0x2e, 0xff, 0x3d, 0xd1, 0xa5, 0xa7, 0x13, 0x5d, 0xfe, + 0xf1, 0x42, 0x97, 0xce, 0x2e, 0x74, 0xe9, 0xc9, 0x85, 0x2e, 0x7d, 0x1d, 0xff, 0x8d, 0xec, 0x64, + 0xb8, 0x1c, 0xdf, 0xf9, 0x27, 0x00, 0x00, 0xff, 0xff, 0x4c, 0xa3, 0x40, 0xfa, 0x63, 0x0e, 0x00, + 0x00, +} + +func (x Aggr) String() string { + s, ok := Aggr_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *SeriesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesRequest) + if !ok { + that2, ok := that.(SeriesRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.MinTime != that1.MinTime { + return false + } + if this.MaxTime != that1.MaxTime { + return false + } + if len(this.Matchers) != len(that1.Matchers) { + return false + } + for i := range this.Matchers { + if !this.Matchers[i].Equal(&that1.Matchers[i]) { + return false + } + } + if this.MaxResolutionWindow != that1.MaxResolutionWindow { + return false + } + if len(this.Aggregates) != len(that1.Aggregates) { + return false + } + for i := range this.Aggregates { + if this.Aggregates[i] != that1.Aggregates[i] { + return false + } + } + if this.PartialResponseDisabled != that1.PartialResponseDisabled { + return false + } + if this.PartialResponseStrategy != that1.PartialResponseStrategy { + return false + } + if this.SkipChunks != that1.SkipChunks { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + if this.Step != that1.Step { + return false + } + if this.Range != that1.Range { + return false + } + if !this.QueryHints.Equal(that1.QueryHints) { + return false + } + if !this.ShardInfo.Equal(that1.ShardInfo) { + return false + } + if len(this.WithoutReplicaLabels) != len(that1.WithoutReplicaLabels) { + return false + } + for i := range this.WithoutReplicaLabels { + if this.WithoutReplicaLabels[i] != that1.WithoutReplicaLabels[i] { + return false + } + } + return true +} +func (this *QueryHints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryHints) + if !ok { + that2, ok := that.(QueryHints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.StepMillis != that1.StepMillis { + return false + } + if !this.Func.Equal(that1.Func) { + return false + } + if !this.Grouping.Equal(that1.Grouping) { + return false + } + if !this.Range.Equal(that1.Range) { + return false + } + return true +} +func (this *ShardInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ShardInfo) + if !ok { + that2, ok := that.(ShardInfo) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.ShardIndex != that1.ShardIndex { + return false + } + if this.TotalShards != that1.TotalShards { + return false + } + if this.By != that1.By { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if this.Labels[i] != that1.Labels[i] { + return false + } + } + return true +} +func (this *Func) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Func) + if !ok { + that2, ok := that.(Func) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Name != that1.Name { + return false + } + return true +} +func (this *Grouping) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Grouping) + if !ok { + that2, ok := that.(Grouping) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.By != that1.By { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if this.Labels[i] != that1.Labels[i] { + return false + } + } + return true +} +func (this *Range) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Range) + if !ok { + that2, ok := that.(Range) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Millis != that1.Millis { + return false + } + return true +} +func (this *SeriesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse) + if !ok { + that2, ok := that.(SeriesResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if that1.Result == nil { + if this.Result != nil { + return false + } + } else if this.Result == nil { + return false + } else if !this.Result.Equal(that1.Result) { + return false + } + return true +} +func (this *SeriesResponse_Series) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse_Series) + if !ok { + that2, ok := that.(SeriesResponse_Series) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Series.Equal(that1.Series) { + return false + } + return true +} +func (this *SeriesResponse_Warning) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse_Warning) + if !ok { + that2, ok := that.(SeriesResponse_Warning) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Warning != that1.Warning { + return false + } + return true +} +func (this *SeriesResponse_Hints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse_Hints) + if !ok { + that2, ok := that.(SeriesResponse_Hints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + return true +} +func (this *LabelNamesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelNamesRequest) + if !ok { + that2, ok := that.(LabelNamesRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.PartialResponseDisabled != that1.PartialResponseDisabled { + return false + } + if this.PartialResponseStrategy != that1.PartialResponseStrategy { + return false + } + if this.Start != that1.Start { + return false + } + if this.End != that1.End { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + if len(this.Matchers) != len(that1.Matchers) { + return false + } + for i := range this.Matchers { + if !this.Matchers[i].Equal(&that1.Matchers[i]) { + return false + } + } + return true +} +func (this *LabelNamesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelNamesResponse) + if !ok { + that2, ok := that.(LabelNamesResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Names) != len(that1.Names) { + return false + } + for i := range this.Names { + if this.Names[i] != that1.Names[i] { + return false + } + } + if len(this.Warnings) != len(that1.Warnings) { + return false + } + for i := range this.Warnings { + if this.Warnings[i] != that1.Warnings[i] { + return false + } + } + if !this.Hints.Equal(that1.Hints) { + return false + } + return true +} +func (this *LabelValuesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelValuesRequest) + if !ok { + that2, ok := that.(LabelValuesRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Label != that1.Label { + return false + } + if this.PartialResponseDisabled != that1.PartialResponseDisabled { + return false + } + if this.PartialResponseStrategy != that1.PartialResponseStrategy { + return false + } + if this.Start != that1.Start { + return false + } + if this.End != that1.End { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + if len(this.Matchers) != len(that1.Matchers) { + return false + } + for i := range this.Matchers { + if !this.Matchers[i].Equal(&that1.Matchers[i]) { + return false + } + } + return true +} +func (this *LabelValuesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelValuesResponse) + if !ok { + that2, ok := that.(LabelValuesResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Values) != len(that1.Values) { + return false + } + for i := range this.Values { + if this.Values[i] != that1.Values[i] { + return false + } + } + if len(this.Warnings) != len(that1.Warnings) { + return false + } + for i := range this.Warnings { + if this.Warnings[i] != that1.Warnings[i] { + return false + } + } + if !this.Hints.Equal(that1.Hints) { + return false + } + return true +} +func (this *SelectRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectRequest) + if !ok { + that2, ok := that.(SelectRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.MinTime != that1.MinTime { + return false + } + if this.MaxTime != that1.MaxTime { + return false + } + if len(this.Matchers) != len(that1.Matchers) { + return false + } + for i := range this.Matchers { + if !this.Matchers[i].Equal(&that1.Matchers[i]) { + return false + } + } + if this.MaxResolutionWindow != that1.MaxResolutionWindow { + return false + } + if len(this.Aggregates) != len(that1.Aggregates) { + return false + } + for i := range this.Aggregates { + if this.Aggregates[i] != that1.Aggregates[i] { + return false + } + } + if this.PartialResponseDisabled != that1.PartialResponseDisabled { + return false + } + if this.PartialResponseStrategy != that1.PartialResponseStrategy { + return false + } + if this.SkipChunks != that1.SkipChunks { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + if this.Step != that1.Step { + return false + } + if this.Range != that1.Range { + return false + } + if !this.QueryHints.Equal(that1.QueryHints) { + return false + } + if !this.ShardInfo.Equal(that1.ShardInfo) { + return false + } + if len(this.WithoutReplicaLabels) != len(that1.WithoutReplicaLabels) { + return false + } + for i := range this.WithoutReplicaLabels { + if this.WithoutReplicaLabels[i] != that1.WithoutReplicaLabels[i] { + return false + } + } + return true +} +func (this *SelectResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectResponse) + if !ok { + that2, ok := that.(SelectResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if that1.Result == nil { + if this.Result != nil { + return false + } + } else if this.Result == nil { + return false + } else if !this.Result.Equal(that1.Result) { + return false + } + return true +} +func (this *SelectResponse_Series) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectResponse_Series) + if !ok { + that2, ok := that.(SelectResponse_Series) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Series.Equal(that1.Series) { + return false + } + return true +} +func (this *SelectResponse_Warning) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectResponse_Warning) + if !ok { + that2, ok := that.(SelectResponse_Warning) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Warning != that1.Warning { + return false + } + return true +} +func (this *SelectResponse_Hints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectResponse_Hints) + if !ok { + that2, ok := that.(SelectResponse_Hints) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Hints.Equal(that1.Hints) { + return false + } + return true +} +func (this *ChunksRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ChunksRequest) + if !ok { + that2, ok := that.(ChunksRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BlockId != that1.BlockId { + return false + } + if len(this.Chunkref) != len(that1.Chunkref) { + return false + } + for i := range this.Chunkref { + if this.Chunkref[i] != that1.Chunkref[i] { + return false + } + } + return true +} +func (this *ChunksResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ChunksResponse) + if !ok { + that2, ok := that.(ChunksResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Chunks) != len(that1.Chunks) { + return false + } + for i := range this.Chunks { + if !this.Chunks[i].Equal(that1.Chunks[i]) { + return false + } + } + return true +} +func (this *SeriesRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 18) + s = append(s, "&storepb.SeriesRequest{") + s = append(s, "MinTime: "+fmt.Sprintf("%#v", this.MinTime)+",\n") + s = append(s, "MaxTime: "+fmt.Sprintf("%#v", this.MaxTime)+",\n") + if this.Matchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.Matchers)) + for i := range vs { + vs[i] = &this.Matchers[i] + } + s = append(s, "Matchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "MaxResolutionWindow: "+fmt.Sprintf("%#v", this.MaxResolutionWindow)+",\n") + s = append(s, "Aggregates: "+fmt.Sprintf("%#v", this.Aggregates)+",\n") + s = append(s, "PartialResponseDisabled: "+fmt.Sprintf("%#v", this.PartialResponseDisabled)+",\n") + s = append(s, "PartialResponseStrategy: "+fmt.Sprintf("%#v", this.PartialResponseStrategy)+",\n") + s = append(s, "SkipChunks: "+fmt.Sprintf("%#v", this.SkipChunks)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + s = append(s, "Step: "+fmt.Sprintf("%#v", this.Step)+",\n") + s = append(s, "Range: "+fmt.Sprintf("%#v", this.Range)+",\n") + if this.QueryHints != nil { + s = append(s, "QueryHints: "+fmt.Sprintf("%#v", this.QueryHints)+",\n") + } + if this.ShardInfo != nil { + s = append(s, "ShardInfo: "+fmt.Sprintf("%#v", this.ShardInfo)+",\n") + } + s = append(s, "WithoutReplicaLabels: "+fmt.Sprintf("%#v", this.WithoutReplicaLabels)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *QueryHints) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&storepb.QueryHints{") + s = append(s, "StepMillis: "+fmt.Sprintf("%#v", this.StepMillis)+",\n") + if this.Func != nil { + s = append(s, "Func: "+fmt.Sprintf("%#v", this.Func)+",\n") + } + if this.Grouping != nil { + s = append(s, "Grouping: "+fmt.Sprintf("%#v", this.Grouping)+",\n") + } + if this.Range != nil { + s = append(s, "Range: "+fmt.Sprintf("%#v", this.Range)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ShardInfo) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&storepb.ShardInfo{") + s = append(s, "ShardIndex: "+fmt.Sprintf("%#v", this.ShardIndex)+",\n") + s = append(s, "TotalShards: "+fmt.Sprintf("%#v", this.TotalShards)+",\n") + s = append(s, "By: "+fmt.Sprintf("%#v", this.By)+",\n") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Func) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&storepb.Func{") + s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Grouping) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&storepb.Grouping{") + s = append(s, "By: "+fmt.Sprintf("%#v", this.By)+",\n") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Range) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&storepb.Range{") + s = append(s, "Millis: "+fmt.Sprintf("%#v", this.Millis)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SeriesResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&storepb.SeriesResponse{") + if this.Result != nil { + s = append(s, "Result: "+fmt.Sprintf("%#v", this.Result)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SeriesResponse_Series) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SeriesResponse_Series{` + + `Series:` + fmt.Sprintf("%#v", this.Series) + `}`}, ", ") + return s +} +func (this *SeriesResponse_Warning) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SeriesResponse_Warning{` + + `Warning:` + fmt.Sprintf("%#v", this.Warning) + `}`}, ", ") + return s +} +func (this *SeriesResponse_Hints) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SeriesResponse_Hints{` + + `Hints:` + fmt.Sprintf("%#v", this.Hints) + `}`}, ", ") + return s +} +func (this *LabelNamesRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 10) + s = append(s, "&storepb.LabelNamesRequest{") + s = append(s, "PartialResponseDisabled: "+fmt.Sprintf("%#v", this.PartialResponseDisabled)+",\n") + s = append(s, "PartialResponseStrategy: "+fmt.Sprintf("%#v", this.PartialResponseStrategy)+",\n") + s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") + s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + if this.Matchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.Matchers)) + for i := range vs { + vs[i] = &this.Matchers[i] + } + s = append(s, "Matchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelNamesResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&storepb.LabelNamesResponse{") + s = append(s, "Names: "+fmt.Sprintf("%#v", this.Names)+",\n") + s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelValuesRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 11) + s = append(s, "&storepb.LabelValuesRequest{") + s = append(s, "Label: "+fmt.Sprintf("%#v", this.Label)+",\n") + s = append(s, "PartialResponseDisabled: "+fmt.Sprintf("%#v", this.PartialResponseDisabled)+",\n") + s = append(s, "PartialResponseStrategy: "+fmt.Sprintf("%#v", this.PartialResponseStrategy)+",\n") + s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") + s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + if this.Matchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.Matchers)) + for i := range vs { + vs[i] = &this.Matchers[i] + } + s = append(s, "Matchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelValuesResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&storepb.LabelValuesResponse{") + s = append(s, "Values: "+fmt.Sprintf("%#v", this.Values)+",\n") + s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SelectRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 18) + s = append(s, "&storepb.SelectRequest{") + s = append(s, "MinTime: "+fmt.Sprintf("%#v", this.MinTime)+",\n") + s = append(s, "MaxTime: "+fmt.Sprintf("%#v", this.MaxTime)+",\n") + if this.Matchers != nil { + vs := make([]*typespb.LabelMatcher, len(this.Matchers)) + for i := range vs { + vs[i] = &this.Matchers[i] + } + s = append(s, "Matchers: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "MaxResolutionWindow: "+fmt.Sprintf("%#v", this.MaxResolutionWindow)+",\n") + s = append(s, "Aggregates: "+fmt.Sprintf("%#v", this.Aggregates)+",\n") + s = append(s, "PartialResponseDisabled: "+fmt.Sprintf("%#v", this.PartialResponseDisabled)+",\n") + s = append(s, "PartialResponseStrategy: "+fmt.Sprintf("%#v", this.PartialResponseStrategy)+",\n") + s = append(s, "SkipChunks: "+fmt.Sprintf("%#v", this.SkipChunks)+",\n") + if this.Hints != nil { + s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") + } + s = append(s, "Step: "+fmt.Sprintf("%#v", this.Step)+",\n") + s = append(s, "Range: "+fmt.Sprintf("%#v", this.Range)+",\n") + if this.QueryHints != nil { + s = append(s, "QueryHints: "+fmt.Sprintf("%#v", this.QueryHints)+",\n") + } + if this.ShardInfo != nil { + s = append(s, "ShardInfo: "+fmt.Sprintf("%#v", this.ShardInfo)+",\n") + } + s = append(s, "WithoutReplicaLabels: "+fmt.Sprintf("%#v", this.WithoutReplicaLabels)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SelectResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&storepb.SelectResponse{") + if this.Result != nil { + s = append(s, "Result: "+fmt.Sprintf("%#v", this.Result)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SelectResponse_Series) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SelectResponse_Series{` + + `Series:` + fmt.Sprintf("%#v", this.Series) + `}`}, ", ") + return s +} +func (this *SelectResponse_Warning) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SelectResponse_Warning{` + + `Warning:` + fmt.Sprintf("%#v", this.Warning) + `}`}, ", ") + return s +} +func (this *SelectResponse_Hints) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SelectResponse_Hints{` + + `Hints:` + fmt.Sprintf("%#v", this.Hints) + `}`}, ", ") + return s +} +func (this *ChunksRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&storepb.ChunksRequest{") + s = append(s, "BlockId: "+fmt.Sprintf("%#v", this.BlockId)+",\n") + s = append(s, "Chunkref: "+fmt.Sprintf("%#v", this.Chunkref)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ChunksResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&storepb.ChunksResponse{") + if this.Chunks != nil { + s = append(s, "Chunks: "+fmt.Sprintf("%#v", this.Chunks)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringRpc(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// StoreClient is the client API for Store service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type StoreClient interface { + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + /// + /// Series should strictly stream full series after series, optionally split by time. This means that a single frame can contain + /// partition of the single series, but once a new series is started to be streamed it means that no more data will + /// be sent for previous one. + /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. + Series(ctx context.Context, in *SeriesRequest, opts ...grpc.CallOption) (Store_SeriesClient, error) + /// LabelNames returns all label names constrained by the given matchers. + LabelNames(ctx context.Context, in *LabelNamesRequest, opts ...grpc.CallOption) (*LabelNamesResponse, error) + /// LabelValues returns all label values for given label name. + LabelValues(ctx context.Context, in *LabelValuesRequest, opts ...grpc.CallOption) (*LabelValuesResponse, error) + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + Select(ctx context.Context, in *SelectRequest, opts ...grpc.CallOption) (Store_SelectClient, error) +} + +type storeClient struct { + cc *grpc.ClientConn +} + +func NewStoreClient(cc *grpc.ClientConn) StoreClient { + return &storeClient{cc} +} + +func (c *storeClient) Series(ctx context.Context, in *SeriesRequest, opts ...grpc.CallOption) (Store_SeriesClient, error) { + stream, err := c.cc.NewStream(ctx, &_Store_serviceDesc.Streams[0], "/storepb.Store/Series", opts...) + if err != nil { + return nil, err + } + x := &storeSeriesClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Store_SeriesClient interface { + Recv() (*SeriesResponse, error) + grpc.ClientStream +} + +type storeSeriesClient struct { + grpc.ClientStream +} + +func (x *storeSeriesClient) Recv() (*SeriesResponse, error) { + m := new(SeriesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *storeClient) LabelNames(ctx context.Context, in *LabelNamesRequest, opts ...grpc.CallOption) (*LabelNamesResponse, error) { + out := new(LabelNamesResponse) + err := c.cc.Invoke(ctx, "/storepb.Store/LabelNames", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *storeClient) LabelValues(ctx context.Context, in *LabelValuesRequest, opts ...grpc.CallOption) (*LabelValuesResponse, error) { + out := new(LabelValuesResponse) + err := c.cc.Invoke(ctx, "/storepb.Store/LabelValues", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *storeClient) Select(ctx context.Context, in *SelectRequest, opts ...grpc.CallOption) (Store_SelectClient, error) { + stream, err := c.cc.NewStream(ctx, &_Store_serviceDesc.Streams[1], "/storepb.Store/Select", opts...) + if err != nil { + return nil, err + } + x := &storeSelectClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Store_SelectClient interface { + Recv() (*SelectResponse, error) + grpc.ClientStream +} + +type storeSelectClient struct { + grpc.ClientStream +} + +func (x *storeSelectClient) Recv() (*SelectResponse, error) { + m := new(SelectResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// StoreServer is the server API for Store service. +type StoreServer interface { + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + /// + /// Series should strictly stream full series after series, optionally split by time. This means that a single frame can contain + /// partition of the single series, but once a new series is started to be streamed it means that no more data will + /// be sent for previous one. + /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. + Series(*SeriesRequest, Store_SeriesServer) error + /// LabelNames returns all label names constrained by the given matchers. + LabelNames(context.Context, *LabelNamesRequest) (*LabelNamesResponse, error) + /// LabelValues returns all label values for given label name. + LabelValues(context.Context, *LabelValuesRequest) (*LabelValuesResponse, error) + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + Select(*SelectRequest, Store_SelectServer) error +} + +// UnimplementedStoreServer can be embedded to have forward compatible implementations. +type UnimplementedStoreServer struct { +} + +func (*UnimplementedStoreServer) Series(req *SeriesRequest, srv Store_SeriesServer) error { + return status.Errorf(codes.Unimplemented, "method Series not implemented") +} +func (*UnimplementedStoreServer) LabelNames(ctx context.Context, req *LabelNamesRequest) (*LabelNamesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LabelNames not implemented") +} +func (*UnimplementedStoreServer) LabelValues(ctx context.Context, req *LabelValuesRequest) (*LabelValuesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method LabelValues not implemented") +} +func (*UnimplementedStoreServer) Select(req *SelectRequest, srv Store_SelectServer) error { + return status.Errorf(codes.Unimplemented, "method Select not implemented") +} + +func RegisterStoreServer(s *grpc.Server, srv StoreServer) { + s.RegisterService(&_Store_serviceDesc, srv) +} + +func _Store_Series_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(SeriesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(StoreServer).Series(m, &storeSeriesServer{stream}) +} + +type Store_SeriesServer interface { + Send(*SeriesResponse) error + grpc.ServerStream +} + +type storeSeriesServer struct { + grpc.ServerStream +} + +func (x *storeSeriesServer) Send(m *SeriesResponse) error { + return x.ServerStream.SendMsg(m) +} + +func _Store_LabelNames_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LabelNamesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StoreServer).LabelNames(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/storepb.Store/LabelNames", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StoreServer).LabelNames(ctx, req.(*LabelNamesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Store_LabelValues_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(LabelValuesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StoreServer).LabelValues(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/storepb.Store/LabelValues", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StoreServer).LabelValues(ctx, req.(*LabelValuesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Store_Select_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(SelectRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(StoreServer).Select(m, &storeSelectServer{stream}) +} + +type Store_SelectServer interface { + Send(*SelectResponse) error + grpc.ServerStream +} + +type storeSelectServer struct { + grpc.ServerStream +} + +func (x *storeSelectServer) Send(m *SelectResponse) error { + return x.ServerStream.SendMsg(m) +} + +var _Store_serviceDesc = grpc.ServiceDesc{ + ServiceName: "storepb.Store", + HandlerType: (*StoreServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "LabelNames", + Handler: _Store_LabelNames_Handler, + }, + { + MethodName: "LabelValues", + Handler: _Store_LabelValues_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "Series", + Handler: _Store_Series_Handler, + ServerStreams: true, + }, + { + StreamName: "Select", + Handler: _Store_Select_Handler, + ServerStreams: true, + }, + }, + Metadata: "rpc.proto", +} + +// ChunkStoreClient is the client API for ChunkStore service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ChunkStoreClient interface { + Chunks(ctx context.Context, opts ...grpc.CallOption) (ChunkStore_ChunksClient, error) +} + +type chunkStoreClient struct { + cc *grpc.ClientConn +} + +func NewChunkStoreClient(cc *grpc.ClientConn) ChunkStoreClient { + return &chunkStoreClient{cc} +} + +func (c *chunkStoreClient) Chunks(ctx context.Context, opts ...grpc.CallOption) (ChunkStore_ChunksClient, error) { + stream, err := c.cc.NewStream(ctx, &_ChunkStore_serviceDesc.Streams[0], "/storepb.ChunkStore/Chunks", opts...) + if err != nil { + return nil, err + } + x := &chunkStoreChunksClient{stream} + return x, nil +} + +type ChunkStore_ChunksClient interface { + Send(*ChunksRequest) error + Recv() (*ChunksResponse, error) + grpc.ClientStream +} + +type chunkStoreChunksClient struct { + grpc.ClientStream +} + +func (x *chunkStoreChunksClient) Send(m *ChunksRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *chunkStoreChunksClient) Recv() (*ChunksResponse, error) { + m := new(ChunksResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// ChunkStoreServer is the server API for ChunkStore service. +type ChunkStoreServer interface { + Chunks(ChunkStore_ChunksServer) error +} + +// UnimplementedChunkStoreServer can be embedded to have forward compatible implementations. +type UnimplementedChunkStoreServer struct { +} + +func (*UnimplementedChunkStoreServer) Chunks(srv ChunkStore_ChunksServer) error { + return status.Errorf(codes.Unimplemented, "method Chunks not implemented") +} + +func RegisterChunkStoreServer(s *grpc.Server, srv ChunkStoreServer) { + s.RegisterService(&_ChunkStore_serviceDesc, srv) +} + +func _ChunkStore_Chunks_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ChunkStoreServer).Chunks(&chunkStoreChunksServer{stream}) +} + +type ChunkStore_ChunksServer interface { + Send(*ChunksResponse) error + Recv() (*ChunksRequest, error) + grpc.ServerStream +} + +type chunkStoreChunksServer struct { + grpc.ServerStream +} + +func (x *chunkStoreChunksServer) Send(m *ChunksResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *chunkStoreChunksServer) Recv() (*ChunksRequest, error) { + m := new(ChunksRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _ChunkStore_serviceDesc = grpc.ServiceDesc{ + ServiceName: "storepb.ChunkStore", + HandlerType: (*ChunkStoreServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Chunks", + Handler: _ChunkStore_Chunks_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "rpc.proto", +} + +func (m *SeriesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.WithoutReplicaLabels) > 0 { + for iNdEx := len(m.WithoutReplicaLabels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.WithoutReplicaLabels[iNdEx]) + copy(dAtA[i:], m.WithoutReplicaLabels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.WithoutReplicaLabels[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if m.ShardInfo != nil { + { + size, err := m.ShardInfo.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x6a + } + if m.QueryHints != nil { + { + size, err := m.QueryHints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + if m.Range != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Range)) + i-- + dAtA[i] = 0x58 + } + if m.Step != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Step)) + i-- + dAtA[i] = 0x50 + } + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + if m.SkipChunks { + i-- + if m.SkipChunks { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } + if m.PartialResponseStrategy != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.PartialResponseStrategy)) + i-- + dAtA[i] = 0x38 + } + if m.PartialResponseDisabled { + i-- + if m.PartialResponseDisabled { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + } + if len(m.Aggregates) > 0 { + dAtA5 := make([]byte, len(m.Aggregates)*10) + var j4 int + for _, num := range m.Aggregates { + for num >= 1<<7 { + dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j4++ + } + dAtA5[j4] = uint8(num) + j4++ + } + i -= j4 + copy(dAtA[i:], dAtA5[:j4]) + i = encodeVarintRpc(dAtA, i, uint64(j4)) + i-- + dAtA[i] = 0x2a + } + if m.MaxResolutionWindow != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MaxResolutionWindow)) + i-- + dAtA[i] = 0x20 + } + if len(m.Matchers) > 0 { + for iNdEx := len(m.Matchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Matchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.MaxTime != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MaxTime)) + i-- + dAtA[i] = 0x10 + } + if m.MinTime != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MinTime)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *QueryHints) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *QueryHints) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Range != nil { + { + size, err := m.Range.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.Grouping != nil { + { + size, err := m.Grouping.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + if m.Func != nil { + { + size, err := m.Func.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.StepMillis != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.StepMillis)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ShardInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ShardInfo) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShardInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Labels[iNdEx]) + copy(dAtA[i:], m.Labels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Labels[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if m.By { + i-- + if m.By { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if m.TotalShards != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.TotalShards)) + i-- + dAtA[i] = 0x10 + } + if m.ShardIndex != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.ShardIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Func) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Func) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Func) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Grouping) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Grouping) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Grouping) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Labels[iNdEx]) + copy(dAtA[i:], m.Labels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Labels[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if m.By { + i-- + if m.By { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Range) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Range) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Range) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Millis != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Millis)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SeriesResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Result != nil { + { + size := m.Result.Size() + i -= size + if _, err := m.Result.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *SeriesResponse_Series) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SeriesResponse_Series) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Series != nil { + { + size, err := m.Series.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *SeriesResponse_Warning) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SeriesResponse_Warning) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Warning) + copy(dAtA[i:], m.Warning) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Warning))) + i-- + dAtA[i] = 0x12 + return len(dAtA) - i, nil +} +func (m *SeriesResponse_Hints) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SeriesResponse_Hints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *LabelNamesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelNamesRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelNamesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Matchers) > 0 { + for iNdEx := len(m.Matchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Matchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + } + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.End != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.End)) + i-- + dAtA[i] = 0x20 + } + if m.Start != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Start)) + i-- + dAtA[i] = 0x18 + } + if m.PartialResponseStrategy != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.PartialResponseStrategy)) + i-- + dAtA[i] = 0x10 + } + if m.PartialResponseDisabled { + i-- + if m.PartialResponseDisabled { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LabelNamesResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelNamesResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelNamesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if len(m.Warnings) > 0 { + for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Warnings[iNdEx]) + copy(dAtA[i:], m.Warnings[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Warnings[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Names) > 0 { + for iNdEx := len(m.Names) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Names[iNdEx]) + copy(dAtA[i:], m.Names[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Names[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *LabelValuesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelValuesRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelValuesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Matchers) > 0 { + for iNdEx := len(m.Matchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Matchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + if m.End != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.End)) + i-- + dAtA[i] = 0x28 + } + if m.Start != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Start)) + i-- + dAtA[i] = 0x20 + } + if m.PartialResponseStrategy != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.PartialResponseStrategy)) + i-- + dAtA[i] = 0x18 + } + if m.PartialResponseDisabled { + i-- + if m.PartialResponseDisabled { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if len(m.Label) > 0 { + i -= len(m.Label) + copy(dAtA[i:], m.Label) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Label))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *LabelValuesResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelValuesResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelValuesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if len(m.Warnings) > 0 { + for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Warnings[iNdEx]) + copy(dAtA[i:], m.Warnings[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Warnings[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Values) > 0 { + for iNdEx := len(m.Values) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Values[iNdEx]) + copy(dAtA[i:], m.Values[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Values[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SelectRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SelectRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SelectRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.WithoutReplicaLabels) > 0 { + for iNdEx := len(m.WithoutReplicaLabels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.WithoutReplicaLabels[iNdEx]) + copy(dAtA[i:], m.WithoutReplicaLabels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.WithoutReplicaLabels[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } + if m.ShardInfo != nil { + { + size, err := m.ShardInfo.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x6a + } + if m.QueryHints != nil { + { + size, err := m.QueryHints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + if m.Range != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Range)) + i-- + dAtA[i] = 0x58 + } + if m.Step != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.Step)) + i-- + dAtA[i] = 0x50 + } + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + if m.SkipChunks { + i-- + if m.SkipChunks { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } + if m.PartialResponseStrategy != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.PartialResponseStrategy)) + i-- + dAtA[i] = 0x38 + } + if m.PartialResponseDisabled { + i-- + if m.PartialResponseDisabled { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + } + if len(m.Aggregates) > 0 { + dAtA19 := make([]byte, len(m.Aggregates)*10) + var j18 int + for _, num := range m.Aggregates { + for num >= 1<<7 { + dAtA19[j18] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j18++ + } + dAtA19[j18] = uint8(num) + j18++ + } + i -= j18 + copy(dAtA[i:], dAtA19[:j18]) + i = encodeVarintRpc(dAtA, i, uint64(j18)) + i-- + dAtA[i] = 0x2a + } + if m.MaxResolutionWindow != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MaxResolutionWindow)) + i-- + dAtA[i] = 0x20 + } + if len(m.Matchers) > 0 { + for iNdEx := len(m.Matchers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Matchers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.MaxTime != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MaxTime)) + i-- + dAtA[i] = 0x10 + } + if m.MinTime != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MinTime)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SelectResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SelectResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SelectResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Result != nil { + { + size := m.Result.Size() + i -= size + if _, err := m.Result.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *SelectResponse_Series) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SelectResponse_Series) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Series != nil { + { + size, err := m.Series.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *SelectResponse_Warning) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SelectResponse_Warning) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Warning) + copy(dAtA[i:], m.Warning) + i = encodeVarintRpc(dAtA, i, uint64(len(m.Warning))) + i-- + dAtA[i] = 0x12 + return len(dAtA) - i, nil +} +func (m *SelectResponse_Hints) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SelectResponse_Hints) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Hints != nil { + { + size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *ChunksRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChunksRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ChunksRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Chunkref) > 0 { + dAtA23 := make([]byte, len(m.Chunkref)*10) + var j22 int + for _, num := range m.Chunkref { + for num >= 1<<7 { + dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j22++ + } + dAtA23[j22] = uint8(num) + j22++ + } + i -= j22 + copy(dAtA[i:], dAtA23[:j22]) + i = encodeVarintRpc(dAtA, i, uint64(j22)) + i-- + dAtA[i] = 0x12 + } + if len(m.BlockId) > 0 { + i -= len(m.BlockId) + copy(dAtA[i:], m.BlockId) + i = encodeVarintRpc(dAtA, i, uint64(len(m.BlockId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ChunksResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChunksResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ChunksResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Chunks) > 0 { + for iNdEx := len(m.Chunks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Chunks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func encodeVarintRpc(dAtA []byte, offset int, v uint64) int { + offset -= sovRpc(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *SeriesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MinTime != 0 { + n += 1 + sovRpc(uint64(m.MinTime)) + } + if m.MaxTime != 0 { + n += 1 + sovRpc(uint64(m.MaxTime)) + } + if len(m.Matchers) > 0 { + for _, e := range m.Matchers { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + if m.MaxResolutionWindow != 0 { + n += 1 + sovRpc(uint64(m.MaxResolutionWindow)) + } + if len(m.Aggregates) > 0 { + l = 0 + for _, e := range m.Aggregates { + l += sovRpc(uint64(e)) + } + n += 1 + sovRpc(uint64(l)) + l + } + if m.PartialResponseDisabled { + n += 2 + } + if m.PartialResponseStrategy != 0 { + n += 1 + sovRpc(uint64(m.PartialResponseStrategy)) + } + if m.SkipChunks { + n += 2 + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.Step != 0 { + n += 1 + sovRpc(uint64(m.Step)) + } + if m.Range != 0 { + n += 1 + sovRpc(uint64(m.Range)) + } + if m.QueryHints != nil { + l = m.QueryHints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.ShardInfo != nil { + l = m.ShardInfo.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.WithoutReplicaLabels) > 0 { + for _, s := range m.WithoutReplicaLabels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *QueryHints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.StepMillis != 0 { + n += 1 + sovRpc(uint64(m.StepMillis)) + } + if m.Func != nil { + l = m.Func.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.Grouping != nil { + l = m.Grouping.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.Range != nil { + l = m.Range.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *ShardInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ShardIndex != 0 { + n += 1 + sovRpc(uint64(m.ShardIndex)) + } + if m.TotalShards != 0 { + n += 1 + sovRpc(uint64(m.TotalShards)) + } + if m.By { + n += 2 + } + if len(m.Labels) > 0 { + for _, s := range m.Labels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *Func) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *Grouping) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.By { + n += 2 + } + if len(m.Labels) > 0 { + for _, s := range m.Labels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *Range) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Millis != 0 { + n += 1 + sovRpc(uint64(m.Millis)) + } + return n +} + +func (m *SeriesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result != nil { + n += m.Result.Size() + } + return n +} + +func (m *SeriesResponse_Series) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Series != nil { + l = m.Series.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} +func (m *SeriesResponse_Warning) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Warning) + n += 1 + l + sovRpc(uint64(l)) + return n +} +func (m *SeriesResponse_Hints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} +func (m *LabelNamesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.PartialResponseDisabled { + n += 2 + } + if m.PartialResponseStrategy != 0 { + n += 1 + sovRpc(uint64(m.PartialResponseStrategy)) + } + if m.Start != 0 { + n += 1 + sovRpc(uint64(m.Start)) + } + if m.End != 0 { + n += 1 + sovRpc(uint64(m.End)) + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.Matchers) > 0 { + for _, e := range m.Matchers { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *LabelNamesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Names) > 0 { + for _, s := range m.Names { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.Warnings) > 0 { + for _, s := range m.Warnings { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *LabelValuesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Label) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if m.PartialResponseDisabled { + n += 2 + } + if m.PartialResponseStrategy != 0 { + n += 1 + sovRpc(uint64(m.PartialResponseStrategy)) + } + if m.Start != 0 { + n += 1 + sovRpc(uint64(m.Start)) + } + if m.End != 0 { + n += 1 + sovRpc(uint64(m.End)) + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.Matchers) > 0 { + for _, e := range m.Matchers { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *LabelValuesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Values) > 0 { + for _, s := range m.Values { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.Warnings) > 0 { + for _, s := range m.Warnings { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *SelectRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MinTime != 0 { + n += 1 + sovRpc(uint64(m.MinTime)) + } + if m.MaxTime != 0 { + n += 1 + sovRpc(uint64(m.MaxTime)) + } + if len(m.Matchers) > 0 { + for _, e := range m.Matchers { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + if m.MaxResolutionWindow != 0 { + n += 1 + sovRpc(uint64(m.MaxResolutionWindow)) + } + if len(m.Aggregates) > 0 { + l = 0 + for _, e := range m.Aggregates { + l += sovRpc(uint64(e)) + } + n += 1 + sovRpc(uint64(l)) + l + } + if m.PartialResponseDisabled { + n += 2 + } + if m.PartialResponseStrategy != 0 { + n += 1 + sovRpc(uint64(m.PartialResponseStrategy)) + } + if m.SkipChunks { + n += 2 + } + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.Step != 0 { + n += 1 + sovRpc(uint64(m.Step)) + } + if m.Range != 0 { + n += 1 + sovRpc(uint64(m.Range)) + } + if m.QueryHints != nil { + l = m.QueryHints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.ShardInfo != nil { + l = m.ShardInfo.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.WithoutReplicaLabels) > 0 { + for _, s := range m.WithoutReplicaLabels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func (m *SelectResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result != nil { + n += m.Result.Size() + } + return n +} + +func (m *SelectResponse_Series) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Series != nil { + l = m.Series.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} +func (m *SelectResponse_Warning) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Warning) + n += 1 + l + sovRpc(uint64(l)) + return n +} +func (m *SelectResponse_Hints) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Hints != nil { + l = m.Hints.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} +func (m *ChunksRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BlockId) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.Chunkref) > 0 { + l = 0 + for _, e := range m.Chunkref { + l += sovRpc(uint64(e)) + } + n += 1 + sovRpc(uint64(l)) + l + } + return n +} + +func (m *ChunksResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Chunks) > 0 { + for _, e := range m.Chunks { + l = e.Size() + n += 1 + l + sovRpc(uint64(l)) + } + } + return n +} + +func sovRpc(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozRpc(x uint64) (n int) { + return sovRpc(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *SeriesRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForMatchers := "[]LabelMatcher{" + for _, f := range this.Matchers { + repeatedStringForMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForMatchers += "}" + s := strings.Join([]string{`&SeriesRequest{`, + `MinTime:` + fmt.Sprintf("%v", this.MinTime) + `,`, + `MaxTime:` + fmt.Sprintf("%v", this.MaxTime) + `,`, + `Matchers:` + repeatedStringForMatchers + `,`, + `MaxResolutionWindow:` + fmt.Sprintf("%v", this.MaxResolutionWindow) + `,`, + `Aggregates:` + fmt.Sprintf("%v", this.Aggregates) + `,`, + `PartialResponseDisabled:` + fmt.Sprintf("%v", this.PartialResponseDisabled) + `,`, + `PartialResponseStrategy:` + fmt.Sprintf("%v", this.PartialResponseStrategy) + `,`, + `SkipChunks:` + fmt.Sprintf("%v", this.SkipChunks) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `Step:` + fmt.Sprintf("%v", this.Step) + `,`, + `Range:` + fmt.Sprintf("%v", this.Range) + `,`, + `QueryHints:` + strings.Replace(this.QueryHints.String(), "QueryHints", "QueryHints", 1) + `,`, + `ShardInfo:` + strings.Replace(this.ShardInfo.String(), "ShardInfo", "ShardInfo", 1) + `,`, + `WithoutReplicaLabels:` + fmt.Sprintf("%v", this.WithoutReplicaLabels) + `,`, + `}`, + }, "") + return s +} +func (this *QueryHints) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryHints{`, + `StepMillis:` + fmt.Sprintf("%v", this.StepMillis) + `,`, + `Func:` + strings.Replace(this.Func.String(), "Func", "Func", 1) + `,`, + `Grouping:` + strings.Replace(this.Grouping.String(), "Grouping", "Grouping", 1) + `,`, + `Range:` + strings.Replace(this.Range.String(), "Range", "Range", 1) + `,`, + `}`, + }, "") + return s +} +func (this *ShardInfo) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ShardInfo{`, + `ShardIndex:` + fmt.Sprintf("%v", this.ShardIndex) + `,`, + `TotalShards:` + fmt.Sprintf("%v", this.TotalShards) + `,`, + `By:` + fmt.Sprintf("%v", this.By) + `,`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `}`, + }, "") + return s +} +func (this *Func) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Func{`, + `Name:` + fmt.Sprintf("%v", this.Name) + `,`, + `}`, + }, "") + return s +} +func (this *Grouping) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Grouping{`, + `By:` + fmt.Sprintf("%v", this.By) + `,`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `}`, + }, "") + return s +} +func (this *Range) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Range{`, + `Millis:` + fmt.Sprintf("%v", this.Millis) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse{`, + `Result:` + fmt.Sprintf("%v", this.Result) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponse_Series) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse_Series{`, + `Series:` + strings.Replace(fmt.Sprintf("%v", this.Series), "Series", "typespb.Series", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponse_Warning) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse_Warning{`, + `Warning:` + fmt.Sprintf("%v", this.Warning) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponse_Hints) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse_Hints{`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `}`, + }, "") + return s +} +func (this *LabelNamesRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForMatchers := "[]LabelMatcher{" + for _, f := range this.Matchers { + repeatedStringForMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForMatchers += "}" + s := strings.Join([]string{`&LabelNamesRequest{`, + `PartialResponseDisabled:` + fmt.Sprintf("%v", this.PartialResponseDisabled) + `,`, + `PartialResponseStrategy:` + fmt.Sprintf("%v", this.PartialResponseStrategy) + `,`, + `Start:` + fmt.Sprintf("%v", this.Start) + `,`, + `End:` + fmt.Sprintf("%v", this.End) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `Matchers:` + repeatedStringForMatchers + `,`, + `}`, + }, "") + return s +} +func (this *LabelNamesResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&LabelNamesResponse{`, + `Names:` + fmt.Sprintf("%v", this.Names) + `,`, + `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `}`, + }, "") + return s +} +func (this *LabelValuesRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForMatchers := "[]LabelMatcher{" + for _, f := range this.Matchers { + repeatedStringForMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForMatchers += "}" + s := strings.Join([]string{`&LabelValuesRequest{`, + `Label:` + fmt.Sprintf("%v", this.Label) + `,`, + `PartialResponseDisabled:` + fmt.Sprintf("%v", this.PartialResponseDisabled) + `,`, + `PartialResponseStrategy:` + fmt.Sprintf("%v", this.PartialResponseStrategy) + `,`, + `Start:` + fmt.Sprintf("%v", this.Start) + `,`, + `End:` + fmt.Sprintf("%v", this.End) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `Matchers:` + repeatedStringForMatchers + `,`, + `}`, + }, "") + return s +} +func (this *LabelValuesResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&LabelValuesResponse{`, + `Values:` + fmt.Sprintf("%v", this.Values) + `,`, + `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SelectRequest) String() string { + if this == nil { + return "nil" + } + repeatedStringForMatchers := "[]LabelMatcher{" + for _, f := range this.Matchers { + repeatedStringForMatchers += fmt.Sprintf("%v", f) + "," + } + repeatedStringForMatchers += "}" + s := strings.Join([]string{`&SelectRequest{`, + `MinTime:` + fmt.Sprintf("%v", this.MinTime) + `,`, + `MaxTime:` + fmt.Sprintf("%v", this.MaxTime) + `,`, + `Matchers:` + repeatedStringForMatchers + `,`, + `MaxResolutionWindow:` + fmt.Sprintf("%v", this.MaxResolutionWindow) + `,`, + `Aggregates:` + fmt.Sprintf("%v", this.Aggregates) + `,`, + `PartialResponseDisabled:` + fmt.Sprintf("%v", this.PartialResponseDisabled) + `,`, + `PartialResponseStrategy:` + fmt.Sprintf("%v", this.PartialResponseStrategy) + `,`, + `SkipChunks:` + fmt.Sprintf("%v", this.SkipChunks) + `,`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `Step:` + fmt.Sprintf("%v", this.Step) + `,`, + `Range:` + fmt.Sprintf("%v", this.Range) + `,`, + `QueryHints:` + strings.Replace(this.QueryHints.String(), "QueryHints", "QueryHints", 1) + `,`, + `ShardInfo:` + strings.Replace(this.ShardInfo.String(), "ShardInfo", "ShardInfo", 1) + `,`, + `WithoutReplicaLabels:` + fmt.Sprintf("%v", this.WithoutReplicaLabels) + `,`, + `}`, + }, "") + return s +} +func (this *SelectResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SelectResponse{`, + `Result:` + fmt.Sprintf("%v", this.Result) + `,`, + `}`, + }, "") + return s +} +func (this *SelectResponse_Series) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SelectResponse_Series{`, + `Series:` + strings.Replace(fmt.Sprintf("%v", this.Series), "SelectedSeries", "typespb.SelectedSeries", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SelectResponse_Warning) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SelectResponse_Warning{`, + `Warning:` + fmt.Sprintf("%v", this.Warning) + `,`, + `}`, + }, "") + return s +} +func (this *SelectResponse_Hints) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SelectResponse_Hints{`, + `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `}`, + }, "") + return s +} +func (this *ChunksRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ChunksRequest{`, + `BlockId:` + fmt.Sprintf("%v", this.BlockId) + `,`, + `Chunkref:` + fmt.Sprintf("%v", this.Chunkref) + `,`, + `}`, + }, "") + return s +} +func (this *ChunksResponse) String() string { + if this == nil { + return "nil" + } + repeatedStringForChunks := "[]*AggrChunk{" + for _, f := range this.Chunks { + repeatedStringForChunks += strings.Replace(fmt.Sprintf("%v", f), "AggrChunk", "typespb.AggrChunk", 1) + "," + } + repeatedStringForChunks += "}" + s := strings.Join([]string{`&ChunksResponse{`, + `Chunks:` + repeatedStringForChunks + `,`, + `}`, + }, "") + return s +} +func valueToStringRpc(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *SeriesRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MinTime", wireType) + } + m.MinTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MinTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxTime", wireType) + } + m.MaxTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Matchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Matchers = append(m.Matchers, typespb.LabelMatcher{}) + if err := m.Matchers[len(m.Matchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxResolutionWindow", wireType) + } + m.MaxResolutionWindow = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxResolutionWindow |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType == 0 { + var v Aggr + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Aggr(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Aggregates = append(m.Aggregates, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.Aggregates) == 0 { + m.Aggregates = make([]Aggr, 0, elementCount) + } + for iNdEx < postIndex { + var v Aggr + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Aggr(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Aggregates = append(m.Aggregates, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Aggregates", wireType) + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseDisabled", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.PartialResponseDisabled = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseStrategy", wireType) + } + m.PartialResponseStrategy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PartialResponseStrategy |= typespb.PartialResponseStrategy(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipChunks", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SkipChunks = bool(v != 0) + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Step", wireType) + } + m.Step = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Step |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) + } + m.Range = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Range |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryHints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.QueryHints == nil { + m.QueryHints = &QueryHints{} + } + if err := m.QueryHints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ShardInfo == nil { + m.ShardInfo = &ShardInfo{} + } + if err := m.ShardInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithoutReplicaLabels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithoutReplicaLabels = append(m.WithoutReplicaLabels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *QueryHints) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: QueryHints: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: QueryHints: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StepMillis", wireType) + } + m.StepMillis = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StepMillis |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Func", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Func == nil { + m.Func = &Func{} + } + if err := m.Func.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Grouping", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Grouping == nil { + m.Grouping = &Grouping{} + } + if err := m.Grouping.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Range == nil { + m.Range = &Range{} + } + if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ShardInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ShardInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardIndex", wireType) + } + m.ShardIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ShardIndex |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalShards", wireType) + } + m.TotalShards = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TotalShards |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field By", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.By = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Func) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Func: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Func: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Grouping) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Grouping: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Grouping: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field By", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.By = bool(v != 0) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Range) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Range: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Range: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Millis", wireType) + } + m.Millis = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Millis |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SeriesResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Series", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &typespb.Series{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SeriesResponse_Series{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warning", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Result = &SeriesResponse_Warning{string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &types.Any{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SeriesResponse_Hints{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelNamesRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelNamesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelNamesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseDisabled", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.PartialResponseDisabled = bool(v != 0) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseStrategy", wireType) + } + m.PartialResponseStrategy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PartialResponseStrategy |= typespb.PartialResponseStrategy(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + m.Start = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Start |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) + } + m.End = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.End |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Matchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Matchers = append(m.Matchers, typespb.LabelMatcher{}) + if err := m.Matchers[len(m.Matchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelNamesResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelNamesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelNamesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Names", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Names = append(m.Names, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelValuesRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelValuesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelValuesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Label", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Label = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseDisabled", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.PartialResponseDisabled = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseStrategy", wireType) + } + m.PartialResponseStrategy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PartialResponseStrategy |= typespb.PartialResponseStrategy(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + m.Start = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Start |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) + } + m.End = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.End |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Matchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Matchers = append(m.Matchers, typespb.LabelMatcher{}) + if err := m.Matchers[len(m.Matchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelValuesResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelValuesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelValuesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SelectRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SelectRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SelectRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MinTime", wireType) + } + m.MinTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MinTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxTime", wireType) + } + m.MaxTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Matchers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Matchers = append(m.Matchers, typespb.LabelMatcher{}) + if err := m.Matchers[len(m.Matchers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxResolutionWindow", wireType) + } + m.MaxResolutionWindow = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxResolutionWindow |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType == 0 { + var v Aggr + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Aggr(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Aggregates = append(m.Aggregates, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.Aggregates) == 0 { + m.Aggregates = make([]Aggr, 0, elementCount) + } + for iNdEx < postIndex { + var v Aggr + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Aggr(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Aggregates = append(m.Aggregates, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Aggregates", wireType) + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseDisabled", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.PartialResponseDisabled = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PartialResponseStrategy", wireType) + } + m.PartialResponseStrategy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PartialResponseStrategy |= typespb.PartialResponseStrategy(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipChunks", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SkipChunks = bool(v != 0) + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Hints == nil { + m.Hints = &types.Any{} + } + if err := m.Hints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Step", wireType) + } + m.Step = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Step |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) + } + m.Range = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Range |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryHints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.QueryHints == nil { + m.QueryHints = &QueryHints{} + } + if err := m.QueryHints.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ShardInfo == nil { + m.ShardInfo = &ShardInfo{} + } + if err := m.ShardInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithoutReplicaLabels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithoutReplicaLabels = append(m.WithoutReplicaLabels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SelectResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SelectResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SelectResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Series", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &typespb.SelectedSeries{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SelectResponse_Series{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warning", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Result = &SelectResponse_Warning{string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &types.Any{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SelectResponse_Hints{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChunksRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ChunksRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChunksRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Chunkref = append(m.Chunkref, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Chunkref) == 0 { + m.Chunkref = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Chunkref = append(m.Chunkref, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Chunkref", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChunksResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ChunksResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChunksResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Chunks = append(m.Chunks, &typespb.AggrChunk{}) + if err := m.Chunks[len(m.Chunks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipRpc(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthRpc + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthRpc + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipRpc(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthRpc + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthRpc = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowRpc = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/storegateway/storepb/rpc.proto b/pkg/storegateway/storepb/rpc.proto new file mode 100644 index 0000000000..decbf7fb7c --- /dev/null +++ b/pkg/storegateway/storepb/rpc.proto @@ -0,0 +1,312 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +syntax = "proto3"; +package storepb; + +import "gogoproto/gogo.proto"; +import "github.com/cortexproject/cortex/pkg/storegateway/typespb/types.proto"; +import "google/protobuf/any.proto"; + +option go_package = "storepb"; + +option (gogoproto.sizer_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +// Do not generate XXX fields to reduce memory footprint and opening a door +// for zero-copy casts to/from prometheus data types. +option (gogoproto.goproto_unkeyed_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.goproto_sizecache_all) = false; + +/// Store represents API against instance that stores XOR encoded values with label set metadata (e.g Prometheus metrics). +service Store { + + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + /// + /// Series should strictly stream full series after series, optionally split by time. This means that a single frame can contain + /// partition of the single series, but once a new series is started to be streamed it means that no more data will + /// be sent for previous one. + /// Series has to be sorted. + /// + /// There is no requirements on chunk sorting, however it is recommended to have chunk sorted by chunk min time. + /// This heavily optimizes the resource usage on Querier / Federated Queries. + rpc Series(SeriesRequest) returns (stream SeriesResponse); + + /// LabelNames returns all label names constrained by the given matchers. + rpc LabelNames(LabelNamesRequest) returns (LabelNamesResponse); + + /// LabelValues returns all label values for given label name. + rpc LabelValues(LabelValuesRequest) returns (LabelValuesResponse); + + /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. + rpc Select(SelectRequest) returns (stream SelectResponse); +} + +message SeriesRequest { + int64 min_time = 1; + int64 max_time = 2; + repeated typespb.LabelMatcher matchers = 3 [(gogoproto.nullable) = false]; + + int64 max_resolution_window = 4; + repeated Aggr aggregates = 5; + + // Deprecated. Use partial_response_strategy instead. + bool partial_response_disabled = 6; + + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + typespb.PartialResponseStrategy partial_response_strategy = 7; + + // skip_chunks controls whether sending chunks or not in series responses. + bool skip_chunks = 8; + + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + google.protobuf.Any hints = 9; + + // Query step size in milliseconds. + // Deprecated: Use query_hints instead. + int64 step = 10; + + // Range vector selector range in milliseconds. + // Deprecated: Use query_hints instead. + int64 range = 11; + + // query_hints are the hints coming from the PromQL engine when + // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. + QueryHints query_hints = 12; + + // shard_info is used by the querier to request a specific + // shard of blocks instead of entire blocks. + ShardInfo shard_info = 13; + + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +} + +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. +message QueryHints { + // Query step size in milliseconds. + int64 step_millis = 1; + + // The surrounding function or aggregation. + Func func = 2; + + // The grouping expression + Grouping grouping = 4; + + // Range vector selector. + Range range = 5; +} + +// ShardInfo are the parameters used to shard series in Stores. +message ShardInfo { + // The index of the current shard. + int64 shard_index = 1; + + // The total number of shards. + int64 total_shards = 2; + + // Group by or without labels. + bool by = 3; + + // Labels on which to partition series. + repeated string labels = 4; +} + +message Func { + // The function or aggregation name + string name = 1; +} + +message Grouping { + // Indicate whether it is without or by. + bool by = 1; + + // List of label names used in the grouping. + repeated string labels = 3; +} + +message Range { + int64 millis = 1; +} + +enum Aggr { + RAW = 0; + COUNT = 1; + SUM = 2; + MIN = 3; + MAX = 4; + COUNTER = 5; +} + +message SeriesResponse { + oneof result { + /// series contains 1 response series. The series labels are sorted by name. + typespb.Series series = 1; + + /// warning is considered an information piece in place of series for warning purposes. + /// It is used to warn store API user about suspicious cases or partial response (if enabled). + string warning = 2; + + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. It's also implementation specific if it's allowed that + /// multiple SeriesResponse frames contain hints for a single Series() request and how should they + /// be handled in such case (ie. merged vs keep the first/last one). + google.protobuf.Any hints = 3; + } +} + +message LabelNamesRequest { + bool partial_response_disabled = 1; + + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + typespb.PartialResponseStrategy partial_response_strategy = 2; + + int64 start = 3; + + int64 end = 4; + + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + google.protobuf.Any hints = 5; + + repeated typespb.LabelMatcher matchers = 6 [(gogoproto.nullable) = false]; +} + +message LabelNamesResponse { + repeated string names = 1; + repeated string warnings = 2; + + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. + google.protobuf.Any hints = 3; + } + + message LabelValuesRequest { + string label = 1; + + bool partial_response_disabled = 2; + + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + typespb.PartialResponseStrategy partial_response_strategy = 3; + + int64 start = 4; + + int64 end = 5; + + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + google.protobuf.Any hints = 6; + + repeated typespb.LabelMatcher matchers = 7 [(gogoproto.nullable) = false]; +} + +message LabelValuesResponse { + repeated string values = 1; + repeated string warnings = 2; + + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. + google.protobuf.Any hints = 3; +} + +message SelectRequest { + int64 min_time = 1; + int64 max_time = 2; + repeated typespb.LabelMatcher matchers = 3 [(gogoproto.nullable) = false]; + + int64 max_resolution_window = 4; + repeated Aggr aggregates = 5; + + // Deprecated. Use partial_response_strategy instead. + bool partial_response_disabled = 6; + + // TODO(bwplotka): Move Thanos components to use strategy instead. Including QueryAPI. + typespb.PartialResponseStrategy partial_response_strategy = 7; + + // skip_chunks controls whether sending chunks or not in series responses. + bool skip_chunks = 8; + + // hints is an opaque data structure that can be used to carry additional information. + // The content of this field and whether it's supported depends on the + // implementation of a specific store. + google.protobuf.Any hints = 9; + + // Query step size in milliseconds. + // Deprecated: Use query_hints instead. + int64 step = 10; + + // Range vector selector range in milliseconds. + // Deprecated: Use query_hints instead. + int64 range = 11; + + // query_hints are the hints coming from the PromQL engine when + // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. + QueryHints query_hints = 12; + + // shard_info is used by the querier to request a specific + // shard of blocks instead of entire blocks. + ShardInfo shard_info = 13; + + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +} + +message SelectResponse { + oneof result { + /// series contains 1 response series. The series labels are sorted by name. + typespb.SelectedSeries series = 1; + + /// warning is considered an information piece in place of series for warning purposes. + /// It is used to warn store API user about suspicious cases or partial response (if enabled). + string warning = 2; + + /// hints is an opaque data structure that can be used to carry additional information from + /// the store. The content of this field and whether it's supported depends on the + /// implementation of a specific store. It's also implementation specific if it's allowed that + /// multiple SeriesResponse frames contain hints for a single Series() request and how should they + /// be handled in such case (ie. merged vs keep the first/last one). + google.protobuf.Any hints = 3; + } +} + +service ChunkStore { + rpc Chunks(stream ChunksRequest) returns (stream ChunksResponse); +} + +message ChunksRequest { + string blockId = 1; + repeated uint64 chunkref = 2; +} + +message ChunksResponse { + repeated typespb.AggrChunk chunks = 1; +} diff --git a/pkg/storegateway/storepb/shard_info.go b/pkg/storegateway/storepb/shard_info.go new file mode 100644 index 0000000000..e69617dd2f --- /dev/null +++ b/pkg/storegateway/storepb/shard_info.go @@ -0,0 +1,107 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storepb + +import ( + "sync" + + "github.com/cespare/xxhash/v2" + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +var sep = []byte{'\xff'} + +type ShardMatcher struct { + buf *[]byte + buffers *sync.Pool + shardingLabelset map[string]struct{} + + isSharded bool + by bool + totalShards int64 + shardIndex int64 +} + +func (s *ShardMatcher) IsSharded() bool { + return s.isSharded +} + +func (s *ShardMatcher) Close() { + if s.buffers != nil { + s.buffers.Put(s.buf) + } +} + +func (s *ShardMatcher) MatchesZLabels(zLabels []labelpb.ZLabel) bool { + // Match all series when query is not sharded + if s == nil || !s.isSharded { + return true + } + + *s.buf = (*s.buf)[:0] + for _, lbl := range zLabels { + if shardByLabel(s.shardingLabelset, lbl, s.by) { + *s.buf = append(*s.buf, lbl.Name...) + *s.buf = append(*s.buf, sep[0]) + *s.buf = append(*s.buf, lbl.Value...) + *s.buf = append(*s.buf, sep[0]) + } + } + + hash := xxhash.Sum64(*s.buf) + return hash%uint64(s.totalShards) == uint64(s.shardIndex) +} + +func (s *ShardMatcher) MatchesLabels(lbls labels.Labels) bool { + return s.MatchesZLabels(labelpb.ZLabelsFromPromLabels(lbls)) +} + +func shardByLabel(labelSet map[string]struct{}, zlabel labelpb.ZLabel, groupingBy bool) bool { + _, shardHasLabel := labelSet[zlabel.Name] + if groupingBy && shardHasLabel { + return true + } + + groupingWithout := !groupingBy + if groupingWithout && !shardHasLabel { + return true + } + + return false +} + +func (m *ShardInfo) Matcher(buffers *sync.Pool) *ShardMatcher { + if m == nil || m.TotalShards < 1 { + return &ShardMatcher{ + isSharded: false, + } + } + + return &ShardMatcher{ + isSharded: true, + buf: buffers.Get().(*[]byte), + buffers: buffers, + shardingLabelset: m.labelSet(), + by: m.By, + totalShards: m.TotalShards, + shardIndex: m.ShardIndex, + } +} + +func (m *ShardInfo) labelSet() map[string]struct{} { + if m == nil { + return nil + } + labelSet := make(map[string]struct{}) + if m == nil || m.Labels == nil { + return labelSet + } + + for _, label := range m.Labels { + labelSet[label] = struct{}{} + } + + return labelSet +} diff --git a/pkg/storegateway/storepb/testutil/client.go b/pkg/storegateway/storepb/testutil/client.go new file mode 100644 index 0000000000..90874842d6 --- /dev/null +++ b/pkg/storegateway/storepb/testutil/client.go @@ -0,0 +1,32 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storetestutil + +import ( + "github.com/prometheus/prometheus/model/labels" + + "github.com/thanos-io/thanos/pkg/info/infopb" + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +type TestClient struct { + storepb.StoreClient + + Name string + + ExtLset []labels.Labels + MinTime, MaxTime int64 + Shardable bool + WithoutReplicaLabelsEnabled bool + IsLocalStore bool + StoreTSDBInfos []infopb.TSDBInfo +} + +func (c TestClient) LabelSets() []labels.Labels { return c.ExtLset } +func (c TestClient) TimeRange() (mint, maxt int64) { return c.MinTime, c.MaxTime } +func (c TestClient) TSDBInfos() []infopb.TSDBInfo { return c.StoreTSDBInfos } +func (c TestClient) SupportsSharding() bool { return c.Shardable } +func (c TestClient) SupportsWithoutReplicaLabels() bool { return c.WithoutReplicaLabelsEnabled } +func (c TestClient) String() string { return c.Name } +func (c TestClient) Addr() (string, bool) { return c.Name, c.IsLocalStore } diff --git a/pkg/storegateway/storepb/testutil/series.go b/pkg/storegateway/storepb/testutil/series.go new file mode 100644 index 0000000000..afffc75631 --- /dev/null +++ b/pkg/storegateway/storepb/testutil/series.go @@ -0,0 +1,366 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storetestutil + +import ( + "context" + "fmt" + "math" + "math/rand" + "os" + "path/filepath" + "runtime" + "sort" + "testing" + "time" + + "github.com/cespare/xxhash" + "github.com/cortexproject/cortex/pkg/storegateway/typespb" + "github.com/efficientgo/core/testutil" + "github.com/go-kit/log" + "github.com/gogo/protobuf/types" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/wlog" + "go.uber.org/atomic" + + "github.com/thanos-io/thanos/pkg/store/hintspb" + "github.com/thanos-io/thanos/pkg/store/labelpb" + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +const ( + // LabelLongSuffix is a label with ~50B in size, to emulate real-world high cardinality. + LabelLongSuffix = "aaaaaaaaaabbbbbbbbbbccccccccccdddddddddd" +) + +func allPostings(ctx context.Context, t testing.TB, ix tsdb.IndexReader) index.Postings { + k, v := index.AllPostingsKey() + p, err := ix.Postings(ctx, k, v) + testutil.Ok(t, err) + return p +} + +type HeadGenOptions struct { + TSDBDir string + SamplesPerSeries, Series int + ScrapeInterval time.Duration + + WithWAL bool + PrependLabels labels.Labels + SkipChunks bool // Skips chunks in returned slice (not in generated head!). + SampleType chunkenc.ValueType + + Random *rand.Rand +} + +func CreateBlockFromHead(t testing.TB, dir string, head *tsdb.Head) ulid.ULID { + compactor, err := tsdb.NewLeveledCompactor(context.Background(), nil, log.NewNopLogger(), []int64{1000000}, nil, nil) + testutil.Ok(t, err) + + testutil.Ok(t, os.MkdirAll(dir, 0777)) + + // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). + // Because of this block intervals are always +1 than the total samples it includes. + ulid, err := compactor.Write(dir, head, head.MinTime(), head.MaxTime()+1, nil) + testutil.Ok(t, err) + return ulid +} + +// CreateHeadWithSeries returns head filled with given samples and same series returned in separate list for assertion purposes. +// Returned series list has "ext1"="1" prepended. Each series looks as follows: +// {foo=bar,i=000001aaaaaaaaaabbbbbbbbbbccccccccccdddddddddd} where number indicate sample number from 0. +// Returned series are framed in the same way as remote read would frame them. +func CreateHeadWithSeries(t testing.TB, j int, opts HeadGenOptions) (*tsdb.Head, []*typespb.Series) { + if opts.SamplesPerSeries < 1 || opts.Series < 1 { + t.Fatal("samples and series has to be 1 or more") + } + if opts.ScrapeInterval == 0 { + opts.ScrapeInterval = 1 * time.Millisecond + } + // Use float type if sample type is not set. + if opts.SampleType == chunkenc.ValNone { + opts.SampleType = chunkenc.ValFloat + } + + fmt.Printf( + "Creating %d %d-sample series with %s interval in %s\n", + opts.Series, + opts.SamplesPerSeries, + opts.ScrapeInterval.String(), + opts.TSDBDir, + ) + + var w *wlog.WL + var err error + if opts.WithWAL { + w, err = wlog.New(nil, nil, filepath.Join(opts.TSDBDir, "wal"), wlog.ParseCompressionType(true, string(wlog.CompressionSnappy))) + testutil.Ok(t, err) + } else { + testutil.Ok(t, os.MkdirAll(filepath.Join(opts.TSDBDir, "wal"), os.ModePerm)) + } + + headOpts := tsdb.DefaultHeadOptions() + headOpts.ChunkDirRoot = opts.TSDBDir + headOpts.EnableNativeHistograms = *atomic.NewBool(true) + h, err := tsdb.NewHead(nil, nil, w, nil, headOpts, nil) + testutil.Ok(t, err) + + app := h.Appender(context.Background()) + for i := 0; i < opts.Series; i++ { + tsLabel := j*opts.Series*opts.SamplesPerSeries + i*opts.SamplesPerSeries + switch opts.SampleType { + case chunkenc.ValFloat: + appendFloatSamples(t, app, tsLabel, opts) + case chunkenc.ValHistogram: + appendHistogramSamples(t, app, tsLabel, opts) + } + } + testutil.Ok(t, app.Commit()) + + return h, ReadSeriesFromBlock(t, h, opts.PrependLabels, opts.SkipChunks) +} + +func ReadSeriesFromBlock(t testing.TB, h tsdb.BlockReader, extLabels labels.Labels, skipChunks bool) []*typespb.Series { + // Use TSDB and get all series for assertion. + chks, err := h.Chunks() + testutil.Ok(t, err) + defer func() { testutil.Ok(t, chks.Close()) }() + + ir, err := h.Index() + testutil.Ok(t, err) + defer func() { testutil.Ok(t, ir.Close()) }() + + var ( + lset labels.Labels + chunkMetas []chunks.Meta + expected = make([]*typespb.Series, 0) + ) + + var builder labels.ScratchBuilder + + all := allPostings(context.TODO(), t, ir) + for all.Next() { + testutil.Ok(t, ir.Series(all.At(), &builder, &chunkMetas)) + lset = builder.Labels() + expected = append(expected, &typespb.Series{Labels: labelpb.ZLabelsFromPromLabels(append(extLabels.Copy(), lset...))}) + + if skipChunks { + continue + } + + for _, c := range chunkMetas { + chEnc, err := chks.Chunk(c) + testutil.Ok(t, err) + + // Open Chunk. + if c.MaxTime == math.MaxInt64 { + c.MaxTime = c.MinTime + int64(chEnc.NumSamples()) - 1 + } + + expected[len(expected)-1].Chunks = append(expected[len(expected)-1].Chunks, typespb.AggrChunk{ + MinTime: c.MinTime, + MaxTime: c.MaxTime, + Raw: &typespb.Chunk{ + Data: chEnc.Bytes(), + Type: typespb.Chunk_Encoding(chEnc.Encoding() - 1), + Hash: xxhash.Sum64(chEnc.Bytes()), + }, + }) + } + } + testutil.Ok(t, all.Err()) + return expected +} + +func appendFloatSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { + ref, err := app.Append( + 0, + labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)), + int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), + opts.Random.Float64(), + ) + testutil.Ok(t, err) + + for is := 1; is < opts.SamplesPerSeries; is++ { + _, err := app.Append(ref, nil, int64(tsLabel+is)*opts.ScrapeInterval.Milliseconds(), opts.Random.Float64()) + testutil.Ok(t, err) + } +} + +func appendHistogramSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { + sample := &histogram.Histogram{ + Schema: 0, + Count: 20, + Sum: -3.1415, + ZeroCount: 12, + ZeroThreshold: 0.001, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 4}, + {Offset: 1, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1}, + } + + ref, err := app.AppendHistogram( + 0, + labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)), + int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), + sample, + nil, + ) + testutil.Ok(t, err) + + for is := 1; is < opts.SamplesPerSeries; is++ { + _, err := app.AppendHistogram(ref, nil, int64(tsLabel+is)*opts.ScrapeInterval.Milliseconds(), sample, nil) + testutil.Ok(t, err) + } +} + +// SeriesServer is test gRPC storeAPI series server. +type SeriesServer struct { + // This field just exist to pseudo-implement the unused methods of the interface. + storepb.Store_SeriesServer + + ctx context.Context + + SeriesSet []*storepb.Series + Warnings []string + HintsSet []*types.Any + + Size int64 +} + +func NewSeriesServer(ctx context.Context) *SeriesServer { + return &SeriesServer{ctx: ctx} +} + +func (s *SeriesServer) Send(r *storepb.SeriesResponse) error { + s.Size += int64(r.Size()) + + if r.GetWarning() != "" { + s.Warnings = append(s.Warnings, r.GetWarning()) + return nil + } + + if r.GetSeries() != nil { + s.SeriesSet = append(s.SeriesSet, r.GetSeries()) + return nil + } + + if r.GetHints() != nil { + s.HintsSet = append(s.HintsSet, r.GetHints()) + return nil + } + // Unsupported field, skip. + return nil +} + +func (s *SeriesServer) Context() context.Context { + return s.ctx +} + +func RunSeriesInterestingCases(t testutil.TB, maxSamples, maxSeries int, f func(t testutil.TB, samplesPerSeries, series int)) { + for _, tc := range []struct { + samplesPerSeries int + series int + }{ + { + samplesPerSeries: 1, + series: maxSeries, + }, + { + samplesPerSeries: maxSamples / (maxSeries / 10), + series: maxSeries / 10, + }, + { + samplesPerSeries: maxSamples, + series: 1, + }, + } { + if ok := t.Run(fmt.Sprintf("%dSeriesWith%dSamples", tc.series, tc.samplesPerSeries), func(t testutil.TB) { + f(t, tc.samplesPerSeries, tc.series) + }); !ok { + return + } + runtime.GC() + } +} + +// SeriesCase represents single test/benchmark case for testing storepb series. +type SeriesCase struct { + Name string + Req *storepb.SeriesRequest + + // Exact expectations are checked only for tests. For benchmarks only length is assured. + ExpectedSeries []*typespb.Series + ExpectedWarnings []string + ExpectedHints []hintspb.SeriesResponseHints + HintsCompareFunc func(t testutil.TB, expected, actual hintspb.SeriesResponseHints) +} + +// TestServerSeries runs tests against given cases. +func TestServerSeries(t testutil.TB, store storepb.StoreServer, cases ...*SeriesCase) { + for _, c := range cases { + t.Run(c.Name, func(t testutil.TB) { + t.ResetTimer() + for i := 0; i < t.N(); i++ { + srv := NewSeriesServer(context.Background()) + testutil.Ok(t, store.Series(c.Req, srv)) + testutil.Equals(t, len(c.ExpectedWarnings), len(srv.Warnings), "%v", srv.Warnings) + testutil.Equals(t, len(c.ExpectedSeries), len(srv.SeriesSet)) + testutil.Equals(t, len(c.ExpectedHints), len(srv.HintsSet)) + + if !t.IsBenchmark() { + if len(c.ExpectedSeries) == 1 { + // For bucketStoreAPI chunks are not sorted within response. TODO: Investigate: Is this fine? + sort.Slice(srv.SeriesSet[0].Chunks, func(i, j int) bool { + return srv.SeriesSet[0].Chunks[i].MinTime < srv.SeriesSet[0].Chunks[j].MinTime + }) + } + + // Huge responses can produce unreadable diffs - make it more human readable. + if len(c.ExpectedSeries) > 4 { + for j := range c.ExpectedSeries { + testutil.Equals(t, c.ExpectedSeries[j].Labels, srv.SeriesSet[j].Labels, "%v series chunks mismatch", j) + + // Check chunks when it is not a skip chunk query + if !c.Req.SkipChunks { + if len(c.ExpectedSeries[j].Chunks) > 20 { + testutil.Equals(t, len(c.ExpectedSeries[j].Chunks), len(srv.SeriesSet[j].Chunks), "%v series chunks number mismatch", j) + } + for ci := range c.ExpectedSeries[j].Chunks { + testutil.Equals(t, c.ExpectedSeries[j].Chunks[ci], srv.SeriesSet[j].Chunks[ci], "%v series chunks mismatch %v", j, ci) + } + } + } + } else { + testutil.Equals(t, c.ExpectedSeries, srv.SeriesSet) + } + + var actualHints []hintspb.SeriesResponseHints + for _, anyHints := range srv.HintsSet { + hints := hintspb.SeriesResponseHints{} + testutil.Ok(t, types.UnmarshalAny(anyHints, &hints)) + actualHints = append(actualHints, hints) + } + testutil.Equals(t, len(c.ExpectedHints), len(actualHints)) + for i, hint := range actualHints { + if c.HintsCompareFunc == nil { + testutil.Equals(t, c.ExpectedHints[i], hint) + } else { + c.HintsCompareFunc(t, c.ExpectedHints[i], hint) + } + } + } + } + }) + } +} diff --git a/pkg/storegateway/storepb/testutil/store_series_client.go b/pkg/storegateway/storepb/testutil/store_series_client.go new file mode 100644 index 0000000000..3647901500 --- /dev/null +++ b/pkg/storegateway/storepb/testutil/store_series_client.go @@ -0,0 +1,51 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storetestutil + +import ( + "context" + "io" + "time" + + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +// StoreSeriesClient is test gRPC storeAPI series client. +type StoreSeriesClient struct { + // This field just exist to pseudo-implement the unused methods of the interface. + storepb.Store_SeriesClient + Ctx context.Context + i int + RespSet []*storepb.SeriesResponse + RespDur time.Duration + SlowSeriesIndex int + + InjectedError error + InjectedErrorIndex int +} + +func (c *StoreSeriesClient) Recv() (*storepb.SeriesResponse, error) { + if c.RespDur != 0 && (c.SlowSeriesIndex == c.i || c.SlowSeriesIndex == 0) { + select { + case <-time.After(c.RespDur): + case <-c.Ctx.Done(): + return nil, c.Ctx.Err() + } + } + if c.InjectedError != nil && (c.InjectedErrorIndex == c.i || c.InjectedErrorIndex == 0) { + return nil, c.InjectedError + } + + if c.i >= len(c.RespSet) { + return nil, io.EOF + } + s := c.RespSet[c.i] + c.i++ + + return s, nil +} + +func (c *StoreSeriesClient) Context() context.Context { + return c.Ctx +} diff --git a/pkg/storegateway/typespb/custom.go b/pkg/storegateway/typespb/custom.go new file mode 100644 index 0000000000..60f0504902 --- /dev/null +++ b/pkg/storegateway/typespb/custom.go @@ -0,0 +1,213 @@ +package typespb + +import ( + bytes "bytes" + "encoding/binary" + fmt "fmt" + "sort" + strconv "strconv" + strings "strings" + + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/store/labelpb" +) + +var PartialResponseStrategyValues = func() []string { + var s []string + for k := range PartialResponseStrategy_value { + s = append(s, k) + } + sort.Strings(s) + return s +}() + +// Compare returns positive 1 if chunk is smaller -1 if larger than b by min time, then max time. +// It returns 0 if chunks are exactly the same. +func (m AggrChunk) Compare(b AggrChunk) int { + if m.MinTime < b.MinTime { + return 1 + } + if m.MinTime > b.MinTime { + return -1 + } + + // Same min time. + if m.MaxTime < b.MaxTime { + return 1 + } + if m.MaxTime > b.MaxTime { + return -1 + } + + // We could use proto.Equal, but we need ordering as well. + for _, cmp := range []func() int{ + func() int { return m.Raw.Compare(b.Raw) }, + func() int { return m.Count.Compare(b.Count) }, + func() int { return m.Sum.Compare(b.Sum) }, + func() int { return m.Min.Compare(b.Min) }, + func() int { return m.Max.Compare(b.Max) }, + func() int { return m.Counter.Compare(b.Counter) }, + } { + if c := cmp(); c == 0 { + continue + } else { + return c + } + } + return 0 +} + +// Compare returns positive 1 if chunk is smaller -1 if larger. +// It returns 0 if chunks are exactly the same. +func (m *Chunk) Compare(b *Chunk) int { + if m == nil && b == nil { + return 0 + } + if b == nil { + return 1 + } + if m == nil { + return -1 + } + + if m.Type < b.Type { + return 1 + } + if m.Type > b.Type { + return -1 + } + return bytes.Compare(m.Data, b.Data) +} + +func (x *PartialResponseStrategy) UnmarshalJSON(entry []byte) error { + fieldStr, err := strconv.Unquote(string(entry)) + if err != nil { + return errors.Wrapf(err, fmt.Sprintf("failed to unqote %v, in order to unmarshal as 'partial_response_strategy'. Possible values are %s", string(entry), strings.Join(PartialResponseStrategyValues, ","))) + } + + if fieldStr == "" { + // NOTE: For Rule default is abort as this is recommended for alerting. + *x = ABORT + return nil + } + + strategy, ok := PartialResponseStrategy_value[strings.ToUpper(fieldStr)] + if !ok { + return errors.Errorf(fmt.Sprintf("failed to unmarshal %v as 'partial_response_strategy'. Possible values are %s", string(entry), strings.Join(PartialResponseStrategyValues, ","))) + } + *x = PartialResponseStrategy(strategy) + return nil +} + +func (x *PartialResponseStrategy) MarshalJSON() ([]byte, error) { + return []byte(strconv.Quote(x.String())), nil +} + +// PromMatchersToMatchers returns proto matchers from Prometheus matchers. +// NOTE: It allocates memory. +func PromMatchersToMatchers(ms ...*labels.Matcher) ([]LabelMatcher, error) { + res := make([]LabelMatcher, 0, len(ms)) + for _, m := range ms { + var t LabelMatcher_Type + + switch m.Type { + case labels.MatchEqual: + t = EQ + case labels.MatchNotEqual: + t = NEQ + case labels.MatchRegexp: + t = RE + case labels.MatchNotRegexp: + t = NRE + default: + return nil, errors.Errorf("unrecognized matcher type %d", m.Type) + } + res = append(res, LabelMatcher{Type: t, Name: m.Name, Value: m.Value}) + } + return res, nil +} + +// MatchersToPromMatchers returns Prometheus matchers from proto matchers. +// NOTE: It allocates memory. +func MatchersToPromMatchers(ms ...LabelMatcher) ([]*labels.Matcher, error) { + res := make([]*labels.Matcher, 0, len(ms)) + for _, m := range ms { + var t labels.MatchType + + switch m.Type { + case EQ: + t = labels.MatchEqual + case NEQ: + t = labels.MatchNotEqual + case RE: + t = labels.MatchRegexp + case NRE: + t = labels.MatchNotRegexp + default: + return nil, errors.Errorf("unrecognized label matcher type %d", m.Type) + } + m, err := labels.NewMatcher(t, m.Name, m.Value) + if err != nil { + return nil, err + } + res = append(res, m) + } + return res, nil +} + +// MatchersToString converts label matchers to string format. +// String should be parsable as a valid PromQL query metric selector. +func MatchersToString(ms ...LabelMatcher) string { + var res string + for i, m := range ms { + res += m.PromString() + if i < len(ms)-1 { + res += ", " + } + } + return "{" + res + "}" +} + +// PromMatchersToString converts prometheus label matchers to string format. +// String should be parsable as a valid PromQL query metric selector. +func PromMatchersToString(ms ...*labels.Matcher) string { + var res string + for i, m := range ms { + res += m.String() + if i < len(ms)-1 { + res += ", " + } + } + return "{" + res + "}" +} + +func (m *LabelMatcher) PromString() string { + return fmt.Sprintf("%s%s%q", m.Name, m.Type.PromString(), m.Value) +} + +func (x LabelMatcher_Type) PromString() string { + typeToStr := map[LabelMatcher_Type]string{ + EQ: "=", + NEQ: "!=", + RE: "=~", + NRE: "!~", + } + if str, ok := typeToStr[x]; ok { + return str + } + panic("unknown match type") +} + +// PromLabels return Prometheus labels.Labels without extra allocation. +func (m *Series) PromLabels() labels.Labels { + return labelpb.ZLabelsToPromLabels(m.Labels) +} + +// XORNumSamples return number of samples. Returns 0 if it's not XOR chunk. +func (m *Chunk) XORNumSamples() int { + if m.Type == XOR { + return int(binary.BigEndian.Uint16(m.Data)) + } + return 0 +} diff --git a/pkg/storegateway/typespb/types.pb.go b/pkg/storegateway/typespb/types.pb.go new file mode 100644 index 0000000000..229fc6c314 --- /dev/null +++ b/pkg/storegateway/typespb/types.pb.go @@ -0,0 +1,2392 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: types.proto + +package typespb + +import ( + bytes "bytes" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + _ "github.com/thanos-io/thanos/pkg/store/labelpb" + github_com_thanos_io_thanos_pkg_store_labelpb "github.com/thanos-io/thanos/pkg/store/labelpb" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strconv "strconv" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// / PartialResponseStrategy controls partial response handling. +type PartialResponseStrategy int32 + +const ( + /// WARN strategy tells server to treat any error that will related to single StoreAPI (e.g missing chunk series because of underlying + /// storeAPI is temporarily not available) as warning which will not fail the whole query (still OK response). + /// Server should produce those as a warnings field in response. + WARN PartialResponseStrategy = 0 + /// ABORT strategy tells server to treat any error that will related to single StoreAPI (e.g missing chunk series because of underlying + /// storeAPI is temporarily not available) as the gRPC error that aborts the query. + /// + /// This is especially useful for any rule/alert evaluations on top of StoreAPI which usually does not tolerate partial + /// errors. + ABORT PartialResponseStrategy = 1 +) + +var PartialResponseStrategy_name = map[int32]string{ + 0: "WARN", + 1: "ABORT", +} + +var PartialResponseStrategy_value = map[string]int32{ + "WARN": 0, + "ABORT": 1, +} + +func (PartialResponseStrategy) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{0} +} + +type Chunk_Encoding int32 + +const ( + XOR Chunk_Encoding = 0 + HISTOGRAM Chunk_Encoding = 1 + FLOAT_HISTOGRAM Chunk_Encoding = 2 +) + +var Chunk_Encoding_name = map[int32]string{ + 0: "XOR", + 1: "HISTOGRAM", + 2: "FLOAT_HISTOGRAM", +} + +var Chunk_Encoding_value = map[string]int32{ + "XOR": 0, + "HISTOGRAM": 1, + "FLOAT_HISTOGRAM": 2, +} + +func (Chunk_Encoding) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{0, 0} +} + +type LabelMatcher_Type int32 + +const ( + EQ LabelMatcher_Type = 0 + NEQ LabelMatcher_Type = 1 + RE LabelMatcher_Type = 2 + NRE LabelMatcher_Type = 3 +) + +var LabelMatcher_Type_name = map[int32]string{ + 0: "EQ", + 1: "NEQ", + 2: "RE", + 3: "NRE", +} + +var LabelMatcher_Type_value = map[string]int32{ + "EQ": 0, + "NEQ": 1, + "RE": 2, + "NRE": 3, +} + +func (LabelMatcher_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{3, 0} +} + +type Chunk struct { + Type Chunk_Encoding `protobuf:"varint,1,opt,name=type,proto3,enum=typespb.Chunk_Encoding" json:"type,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Hash uint64 `protobuf:"varint,3,opt,name=hash,proto3" json:"hash,omitempty"` +} + +func (m *Chunk) Reset() { *m = Chunk{} } +func (*Chunk) ProtoMessage() {} +func (*Chunk) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{0} +} +func (m *Chunk) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Chunk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Chunk.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Chunk) XXX_Merge(src proto.Message) { + xxx_messageInfo_Chunk.Merge(m, src) +} +func (m *Chunk) XXX_Size() int { + return m.Size() +} +func (m *Chunk) XXX_DiscardUnknown() { + xxx_messageInfo_Chunk.DiscardUnknown(m) +} + +var xxx_messageInfo_Chunk proto.InternalMessageInfo + +type Series struct { + Labels []github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel" json:"labels"` + Chunks []AggrChunk `protobuf:"bytes,2,rep,name=chunks,proto3" json:"chunks"` +} + +func (m *Series) Reset() { *m = Series{} } +func (*Series) ProtoMessage() {} +func (*Series) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{1} +} +func (m *Series) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Series) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Series.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Series) XXX_Merge(src proto.Message) { + xxx_messageInfo_Series.Merge(m, src) +} +func (m *Series) XXX_Size() int { + return m.Size() +} +func (m *Series) XXX_DiscardUnknown() { + xxx_messageInfo_Series.DiscardUnknown(m) +} + +var xxx_messageInfo_Series proto.InternalMessageInfo + +type AggrChunk struct { + MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` + MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` + Raw *Chunk `protobuf:"bytes,3,opt,name=raw,proto3" json:"raw,omitempty"` + Count *Chunk `protobuf:"bytes,4,opt,name=count,proto3" json:"count,omitempty"` + Sum *Chunk `protobuf:"bytes,5,opt,name=sum,proto3" json:"sum,omitempty"` + Min *Chunk `protobuf:"bytes,6,opt,name=min,proto3" json:"min,omitempty"` + Max *Chunk `protobuf:"bytes,7,opt,name=max,proto3" json:"max,omitempty"` + Counter *Chunk `protobuf:"bytes,8,opt,name=counter,proto3" json:"counter,omitempty"` +} + +func (m *AggrChunk) Reset() { *m = AggrChunk{} } +func (*AggrChunk) ProtoMessage() {} +func (*AggrChunk) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{2} +} +func (m *AggrChunk) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AggrChunk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_AggrChunk.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *AggrChunk) XXX_Merge(src proto.Message) { + xxx_messageInfo_AggrChunk.Merge(m, src) +} +func (m *AggrChunk) XXX_Size() int { + return m.Size() +} +func (m *AggrChunk) XXX_DiscardUnknown() { + xxx_messageInfo_AggrChunk.DiscardUnknown(m) +} + +var xxx_messageInfo_AggrChunk proto.InternalMessageInfo + +// Matcher specifies a rule, which can match or set of labels or not. +type LabelMatcher struct { + Type LabelMatcher_Type `protobuf:"varint,1,opt,name=type,proto3,enum=typespb.LabelMatcher_Type" json:"type,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` +} + +func (m *LabelMatcher) Reset() { *m = LabelMatcher{} } +func (*LabelMatcher) ProtoMessage() {} +func (*LabelMatcher) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{3} +} +func (m *LabelMatcher) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelMatcher) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LabelMatcher.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *LabelMatcher) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelMatcher.Merge(m, src) +} +func (m *LabelMatcher) XXX_Size() int { + return m.Size() +} +func (m *LabelMatcher) XXX_DiscardUnknown() { + xxx_messageInfo_LabelMatcher.DiscardUnknown(m) +} + +var xxx_messageInfo_LabelMatcher proto.InternalMessageInfo + +type SelectedSeries struct { + BlockId string `protobuf:"bytes,1,opt,name=blockId,proto3" json:"blockId,omitempty"` + Labels []github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel `protobuf:"bytes,2,rep,name=labels,proto3,customtype=github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel" json:"labels"` + Chunks []*ChunkMeta `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` +} + +func (m *SelectedSeries) Reset() { *m = SelectedSeries{} } +func (*SelectedSeries) ProtoMessage() {} +func (*SelectedSeries) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{4} +} +func (m *SelectedSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SelectedSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SelectedSeries.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SelectedSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_SelectedSeries.Merge(m, src) +} +func (m *SelectedSeries) XXX_Size() int { + return m.Size() +} +func (m *SelectedSeries) XXX_DiscardUnknown() { + xxx_messageInfo_SelectedSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_SelectedSeries proto.InternalMessageInfo + +type ChunkMeta struct { + Mint int64 `protobuf:"varint,1,opt,name=mint,proto3" json:"mint,omitempty"` + Maxt int64 `protobuf:"varint,2,opt,name=maxt,proto3" json:"maxt,omitempty"` + Ref uint64 `protobuf:"varint,3,opt,name=ref,proto3" json:"ref,omitempty"` +} + +func (m *ChunkMeta) Reset() { *m = ChunkMeta{} } +func (*ChunkMeta) ProtoMessage() {} +func (*ChunkMeta) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{5} +} +func (m *ChunkMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChunkMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ChunkMeta.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ChunkMeta) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChunkMeta.Merge(m, src) +} +func (m *ChunkMeta) XXX_Size() int { + return m.Size() +} +func (m *ChunkMeta) XXX_DiscardUnknown() { + xxx_messageInfo_ChunkMeta.DiscardUnknown(m) +} + +var xxx_messageInfo_ChunkMeta proto.InternalMessageInfo + +func init() { + proto.RegisterEnum("typespb.PartialResponseStrategy", PartialResponseStrategy_name, PartialResponseStrategy_value) + proto.RegisterEnum("typespb.Chunk_Encoding", Chunk_Encoding_name, Chunk_Encoding_value) + proto.RegisterEnum("typespb.LabelMatcher_Type", LabelMatcher_Type_name, LabelMatcher_Type_value) + proto.RegisterType((*Chunk)(nil), "typespb.Chunk") + proto.RegisterType((*Series)(nil), "typespb.Series") + proto.RegisterType((*AggrChunk)(nil), "typespb.AggrChunk") + proto.RegisterType((*LabelMatcher)(nil), "typespb.LabelMatcher") + proto.RegisterType((*SelectedSeries)(nil), "typespb.SelectedSeries") + proto.RegisterType((*ChunkMeta)(nil), "typespb.ChunkMeta") +} + +func init() { proto.RegisterFile("types.proto", fileDescriptor_d938547f84707355) } + +var fileDescriptor_d938547f84707355 = []byte{ + // 676 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x54, 0x3f, 0x6f, 0xd3, 0x40, + 0x1c, 0xf5, 0xd9, 0xce, 0xbf, 0x5f, 0xff, 0x60, 0x1d, 0x95, 0x6a, 0x3a, 0x5c, 0xa3, 0x88, 0x21, + 0x2a, 0xc2, 0x41, 0x05, 0x09, 0x21, 0xb1, 0x24, 0x28, 0x40, 0xa5, 0xb6, 0xa1, 0x97, 0x48, 0xa0, + 0x2e, 0xd5, 0xc5, 0x39, 0x1c, 0xab, 0xb1, 0x1d, 0xd9, 0x17, 0x48, 0x37, 0x3e, 0x02, 0x62, 0x43, + 0x0c, 0xac, 0x7c, 0x07, 0xbe, 0x40, 0xc6, 0x8e, 0x15, 0x43, 0x45, 0xdc, 0x85, 0xb1, 0x1f, 0x01, + 0xf9, 0xec, 0xa4, 0x2d, 0x0a, 0x03, 0x03, 0xdb, 0xbb, 0xf7, 0xde, 0xfd, 0xce, 0xf7, 0xf4, 0x7c, + 0xb0, 0x24, 0x4e, 0x86, 0x3c, 0xb2, 0x86, 0x61, 0x20, 0x02, 0x5c, 0x90, 0x8b, 0x61, 0x77, 0x63, + 0xcd, 0x09, 0x9c, 0x40, 0x72, 0xb5, 0x04, 0xa5, 0xf2, 0xc6, 0x13, 0xc7, 0x15, 0xfd, 0x51, 0xd7, + 0xb2, 0x03, 0xaf, 0x26, 0xfa, 0xcc, 0x0f, 0xa2, 0xfb, 0x6e, 0x90, 0xa1, 0xda, 0xf0, 0xd8, 0xa9, + 0x45, 0x22, 0x08, 0x79, 0x6d, 0xc0, 0xba, 0x7c, 0x30, 0xec, 0xd6, 0xae, 0x4d, 0xae, 0x7c, 0x41, + 0x90, 0x7b, 0xd6, 0x1f, 0xf9, 0xc7, 0xf8, 0x1e, 0xe8, 0x89, 0x60, 0xa2, 0x32, 0xaa, 0xae, 0x6e, + 0xaf, 0x5b, 0xd9, 0x91, 0x96, 0x54, 0xad, 0xa6, 0x6f, 0x07, 0x3d, 0xd7, 0x77, 0xa8, 0x34, 0x61, + 0x0c, 0x7a, 0x8f, 0x09, 0x66, 0xaa, 0x65, 0x54, 0x5d, 0xa6, 0x12, 0x63, 0x13, 0xf4, 0x3e, 0x8b, + 0xfa, 0xa6, 0x56, 0x46, 0x55, 0xbd, 0xa1, 0x4f, 0xce, 0x37, 0x11, 0x95, 0x4c, 0xe5, 0x31, 0x14, + 0x67, 0xfb, 0x71, 0x01, 0xb4, 0x37, 0x2d, 0x6a, 0x28, 0x78, 0x05, 0x4a, 0x2f, 0x77, 0xda, 0x9d, + 0xd6, 0x0b, 0x5a, 0xdf, 0x33, 0x10, 0xbe, 0x0d, 0xb7, 0x9e, 0xef, 0xb6, 0xea, 0x9d, 0xa3, 0x2b, + 0x52, 0xad, 0x7c, 0x45, 0x90, 0x6f, 0xf3, 0xd0, 0xe5, 0x11, 0xb6, 0x21, 0x2f, 0xbf, 0x3f, 0x32, + 0x51, 0x59, 0xab, 0x2e, 0x6d, 0xaf, 0x58, 0xe9, 0xfd, 0xac, 0xdd, 0x84, 0x6d, 0x3c, 0x9d, 0x9c, + 0x6f, 0x2a, 0x3f, 0xce, 0x37, 0x1f, 0xfd, 0x53, 0x14, 0xd6, 0xa1, 0xdc, 0x4d, 0xb3, 0xd1, 0xf8, + 0x01, 0xe4, 0xed, 0xe4, 0xba, 0x91, 0xa9, 0xca, 0x43, 0xf0, 0x3c, 0x85, 0xba, 0xe3, 0x84, 0x32, + 0x09, 0x79, 0x31, 0x85, 0x66, 0xbe, 0xca, 0x67, 0x15, 0x4a, 0x73, 0x0d, 0xdf, 0x81, 0xa2, 0xe7, + 0xfa, 0x47, 0xc2, 0xf5, 0xd2, 0x1c, 0x35, 0x5a, 0xf0, 0x5c, 0xbf, 0xe3, 0x7a, 0x5c, 0x4a, 0x6c, + 0x9c, 0x4a, 0x6a, 0x26, 0xb1, 0xb1, 0x94, 0xca, 0xa0, 0x85, 0xec, 0xbd, 0xcc, 0x6d, 0x69, 0x7b, + 0xf5, 0x66, 0xf0, 0x34, 0x91, 0xf0, 0x5d, 0xc8, 0xd9, 0xc1, 0xc8, 0x17, 0xa6, 0xbe, 0xd0, 0x93, + 0x8a, 0xc9, 0x9c, 0x68, 0xe4, 0x99, 0xb9, 0xc5, 0x73, 0xa2, 0x91, 0x97, 0x38, 0x3c, 0xd7, 0x37, + 0xf3, 0x8b, 0x1d, 0x9e, 0xeb, 0x4b, 0x07, 0x1b, 0x9b, 0x85, 0xbf, 0x38, 0xd8, 0x18, 0x57, 0xa1, + 0x20, 0x8f, 0xe3, 0xa1, 0x59, 0x5c, 0xe8, 0x9a, 0xc9, 0x95, 0x4f, 0x08, 0x96, 0x65, 0xbe, 0x7b, + 0x4c, 0xd8, 0x7d, 0x1e, 0x62, 0xeb, 0x46, 0xc5, 0x36, 0xe6, 0xfb, 0xae, 0x9b, 0xac, 0xce, 0xc9, + 0x90, 0x5f, 0xb5, 0xcc, 0x67, 0x59, 0x5e, 0x25, 0x2a, 0x31, 0x5e, 0x83, 0xdc, 0x3b, 0x36, 0x18, + 0x71, 0x19, 0x57, 0x89, 0xa6, 0x8b, 0x4a, 0x15, 0xf4, 0x64, 0x1f, 0xce, 0x83, 0xda, 0x3c, 0x30, + 0x94, 0xa4, 0x65, 0xfb, 0xcd, 0x03, 0x03, 0x25, 0x04, 0x6d, 0x1a, 0xaa, 0x24, 0x68, 0xd3, 0xd0, + 0x2a, 0xdf, 0x11, 0xac, 0xb6, 0xf9, 0x80, 0xdb, 0x82, 0xf7, 0xb2, 0x6a, 0x99, 0x50, 0xe8, 0x0e, + 0x02, 0xfb, 0x78, 0xa7, 0x27, 0xbf, 0xac, 0x44, 0x67, 0xcb, 0x6b, 0xa5, 0x53, 0xff, 0x5f, 0xe9, + 0xb6, 0xe6, 0xa5, 0xd3, 0xfe, 0x28, 0x9d, 0xcc, 0x73, 0x8f, 0x0b, 0x36, 0xaf, 0x5b, 0x13, 0x4a, + 0x73, 0x32, 0x89, 0xc7, 0x73, 0x7d, 0x91, 0x35, 0x4d, 0x62, 0xc9, 0xb1, 0xb1, 0xc8, 0x2a, 0x26, + 0x31, 0x36, 0x40, 0x0b, 0xf9, 0xdb, 0xf4, 0xbf, 0xa4, 0x09, 0xdc, 0xb2, 0x60, 0xfd, 0x15, 0x0b, + 0x85, 0xcb, 0x06, 0x94, 0x47, 0xc3, 0xc0, 0x8f, 0x78, 0x5b, 0x84, 0x4c, 0x70, 0xe7, 0x04, 0x17, + 0x41, 0x7f, 0x5d, 0xa7, 0xfb, 0x86, 0x82, 0x4b, 0x90, 0xab, 0x37, 0x5a, 0xb4, 0x63, 0xa0, 0x46, + 0x7d, 0x32, 0x25, 0xca, 0xe9, 0x94, 0x28, 0x67, 0x53, 0xa2, 0x5c, 0x4e, 0x09, 0xfa, 0x10, 0x13, + 0xf4, 0x2d, 0x26, 0x68, 0x12, 0x13, 0x74, 0x1a, 0x13, 0xf4, 0x33, 0x26, 0xe8, 0x57, 0x4c, 0x94, + 0xcb, 0x98, 0xa0, 0x8f, 0x17, 0x44, 0x39, 0xbd, 0x20, 0xca, 0xd9, 0x05, 0x51, 0x0e, 0x67, 0x2f, + 0x57, 0x37, 0x2f, 0xdf, 0x9b, 0x87, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x92, 0x96, 0x83, 0xf5, + 0xd8, 0x04, 0x00, 0x00, +} + +func (x PartialResponseStrategy) String() string { + s, ok := PartialResponseStrategy_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (x Chunk_Encoding) String() string { + s, ok := Chunk_Encoding_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (x LabelMatcher_Type) String() string { + s, ok := LabelMatcher_Type_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *Chunk) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Chunk) + if !ok { + that2, ok := that.(Chunk) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if !bytes.Equal(this.Data, that1.Data) { + return false + } + if this.Hash != that1.Hash { + return false + } + return true +} +func (this *Series) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Series) + if !ok { + that2, ok := that.(Series) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if !this.Labels[i].Equal(that1.Labels[i]) { + return false + } + } + if len(this.Chunks) != len(that1.Chunks) { + return false + } + for i := range this.Chunks { + if !this.Chunks[i].Equal(&that1.Chunks[i]) { + return false + } + } + return true +} +func (this *AggrChunk) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*AggrChunk) + if !ok { + that2, ok := that.(AggrChunk) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.MinTime != that1.MinTime { + return false + } + if this.MaxTime != that1.MaxTime { + return false + } + if !this.Raw.Equal(that1.Raw) { + return false + } + if !this.Count.Equal(that1.Count) { + return false + } + if !this.Sum.Equal(that1.Sum) { + return false + } + if !this.Min.Equal(that1.Min) { + return false + } + if !this.Max.Equal(that1.Max) { + return false + } + if !this.Counter.Equal(that1.Counter) { + return false + } + return true +} +func (this *LabelMatcher) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*LabelMatcher) + if !ok { + that2, ok := that.(LabelMatcher) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if this.Name != that1.Name { + return false + } + if this.Value != that1.Value { + return false + } + return true +} +func (this *SelectedSeries) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SelectedSeries) + if !ok { + that2, ok := that.(SelectedSeries) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BlockId != that1.BlockId { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if !this.Labels[i].Equal(that1.Labels[i]) { + return false + } + } + if len(this.Chunks) != len(that1.Chunks) { + return false + } + for i := range this.Chunks { + if !this.Chunks[i].Equal(that1.Chunks[i]) { + return false + } + } + return true +} +func (this *ChunkMeta) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ChunkMeta) + if !ok { + that2, ok := that.(ChunkMeta) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Mint != that1.Mint { + return false + } + if this.Maxt != that1.Maxt { + return false + } + if this.Ref != that1.Ref { + return false + } + return true +} +func (this *Chunk) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&typespb.Chunk{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + s = append(s, "Data: "+fmt.Sprintf("%#v", this.Data)+",\n") + s = append(s, "Hash: "+fmt.Sprintf("%#v", this.Hash)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Series) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&typespb.Series{") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + if this.Chunks != nil { + vs := make([]*AggrChunk, len(this.Chunks)) + for i := range vs { + vs[i] = &this.Chunks[i] + } + s = append(s, "Chunks: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *AggrChunk) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 12) + s = append(s, "&typespb.AggrChunk{") + s = append(s, "MinTime: "+fmt.Sprintf("%#v", this.MinTime)+",\n") + s = append(s, "MaxTime: "+fmt.Sprintf("%#v", this.MaxTime)+",\n") + if this.Raw != nil { + s = append(s, "Raw: "+fmt.Sprintf("%#v", this.Raw)+",\n") + } + if this.Count != nil { + s = append(s, "Count: "+fmt.Sprintf("%#v", this.Count)+",\n") + } + if this.Sum != nil { + s = append(s, "Sum: "+fmt.Sprintf("%#v", this.Sum)+",\n") + } + if this.Min != nil { + s = append(s, "Min: "+fmt.Sprintf("%#v", this.Min)+",\n") + } + if this.Max != nil { + s = append(s, "Max: "+fmt.Sprintf("%#v", this.Max)+",\n") + } + if this.Counter != nil { + s = append(s, "Counter: "+fmt.Sprintf("%#v", this.Counter)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *LabelMatcher) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&typespb.LabelMatcher{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") + s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SelectedSeries) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&typespb.SelectedSeries{") + s = append(s, "BlockId: "+fmt.Sprintf("%#v", this.BlockId)+",\n") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + if this.Chunks != nil { + s = append(s, "Chunks: "+fmt.Sprintf("%#v", this.Chunks)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ChunkMeta) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&typespb.ChunkMeta{") + s = append(s, "Mint: "+fmt.Sprintf("%#v", this.Mint)+",\n") + s = append(s, "Maxt: "+fmt.Sprintf("%#v", this.Maxt)+",\n") + s = append(s, "Ref: "+fmt.Sprintf("%#v", this.Ref)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringTypes(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *Chunk) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Chunk) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Chunk) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Hash != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Hash)) + i-- + dAtA[i] = 0x18 + } + if len(m.Data) > 0 { + i -= len(m.Data) + copy(dAtA[i:], m.Data) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Data))) + i-- + dAtA[i] = 0x12 + } + if m.Type != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Series) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Series) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Series) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Chunks) > 0 { + for iNdEx := len(m.Chunks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Chunks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *AggrChunk) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AggrChunk) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *AggrChunk) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Counter != nil { + { + size, err := m.Counter.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + if m.Max != nil { + { + size, err := m.Max.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + if m.Min != nil { + { + size, err := m.Min.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + if m.Sum != nil { + { + size, err := m.Sum.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.Count != nil { + { + size, err := m.Count.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + if m.Raw != nil { + { + size, err := m.Raw.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.MaxTime != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.MaxTime)) + i-- + dAtA[i] = 0x10 + } + if m.MinTime != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.MinTime)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LabelMatcher) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LabelMatcher) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LabelMatcher) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x1a + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0x12 + } + if m.Type != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SelectedSeries) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SelectedSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SelectedSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Chunks) > 0 { + for iNdEx := len(m.Chunks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Chunks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.BlockId) > 0 { + i -= len(m.BlockId) + copy(dAtA[i:], m.BlockId) + i = encodeVarintTypes(dAtA, i, uint64(len(m.BlockId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ChunkMeta) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChunkMeta) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ChunkMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Ref != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Ref)) + i-- + dAtA[i] = 0x18 + } + if m.Maxt != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Maxt)) + i-- + dAtA[i] = 0x10 + } + if m.Mint != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Mint)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Chunk) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovTypes(uint64(m.Type)) + } + l = len(m.Data) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.Hash != 0 { + n += 1 + sovTypes(uint64(m.Hash)) + } + return n +} + +func (m *Series) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Chunks) > 0 { + for _, e := range m.Chunks { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func (m *AggrChunk) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MinTime != 0 { + n += 1 + sovTypes(uint64(m.MinTime)) + } + if m.MaxTime != 0 { + n += 1 + sovTypes(uint64(m.MaxTime)) + } + if m.Raw != nil { + l = m.Raw.Size() + n += 1 + l + sovTypes(uint64(l)) + } + if m.Count != nil { + l = m.Count.Size() + n += 1 + l + sovTypes(uint64(l)) + } + if m.Sum != nil { + l = m.Sum.Size() + n += 1 + l + sovTypes(uint64(l)) + } + if m.Min != nil { + l = m.Min.Size() + n += 1 + l + sovTypes(uint64(l)) + } + if m.Max != nil { + l = m.Max.Size() + n += 1 + l + sovTypes(uint64(l)) + } + if m.Counter != nil { + l = m.Counter.Size() + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + +func (m *LabelMatcher) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovTypes(uint64(m.Type)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + +func (m *SelectedSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BlockId) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Chunks) > 0 { + for _, e := range m.Chunks { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func (m *ChunkMeta) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Mint != 0 { + n += 1 + sovTypes(uint64(m.Mint)) + } + if m.Maxt != 0 { + n += 1 + sovTypes(uint64(m.Maxt)) + } + if m.Ref != 0 { + n += 1 + sovTypes(uint64(m.Ref)) + } + return n +} + +func sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *Chunk) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Chunk{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `Data:` + fmt.Sprintf("%v", this.Data) + `,`, + `Hash:` + fmt.Sprintf("%v", this.Hash) + `,`, + `}`, + }, "") + return s +} +func (this *Series) String() string { + if this == nil { + return "nil" + } + repeatedStringForChunks := "[]AggrChunk{" + for _, f := range this.Chunks { + repeatedStringForChunks += strings.Replace(strings.Replace(f.String(), "AggrChunk", "AggrChunk", 1), `&`, ``, 1) + "," + } + repeatedStringForChunks += "}" + s := strings.Join([]string{`&Series{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `Chunks:` + repeatedStringForChunks + `,`, + `}`, + }, "") + return s +} +func (this *AggrChunk) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&AggrChunk{`, + `MinTime:` + fmt.Sprintf("%v", this.MinTime) + `,`, + `MaxTime:` + fmt.Sprintf("%v", this.MaxTime) + `,`, + `Raw:` + strings.Replace(this.Raw.String(), "Chunk", "Chunk", 1) + `,`, + `Count:` + strings.Replace(this.Count.String(), "Chunk", "Chunk", 1) + `,`, + `Sum:` + strings.Replace(this.Sum.String(), "Chunk", "Chunk", 1) + `,`, + `Min:` + strings.Replace(this.Min.String(), "Chunk", "Chunk", 1) + `,`, + `Max:` + strings.Replace(this.Max.String(), "Chunk", "Chunk", 1) + `,`, + `Counter:` + strings.Replace(this.Counter.String(), "Chunk", "Chunk", 1) + `,`, + `}`, + }, "") + return s +} +func (this *LabelMatcher) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&LabelMatcher{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `Name:` + fmt.Sprintf("%v", this.Name) + `,`, + `Value:` + fmt.Sprintf("%v", this.Value) + `,`, + `}`, + }, "") + return s +} +func (this *SelectedSeries) String() string { + if this == nil { + return "nil" + } + repeatedStringForChunks := "[]*ChunkMeta{" + for _, f := range this.Chunks { + repeatedStringForChunks += strings.Replace(f.String(), "ChunkMeta", "ChunkMeta", 1) + "," + } + repeatedStringForChunks += "}" + s := strings.Join([]string{`&SelectedSeries{`, + `BlockId:` + fmt.Sprintf("%v", this.BlockId) + `,`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `Chunks:` + repeatedStringForChunks + `,`, + `}`, + }, "") + return s +} +func (this *ChunkMeta) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ChunkMeta{`, + `Mint:` + fmt.Sprintf("%v", this.Mint) + `,`, + `Maxt:` + fmt.Sprintf("%v", this.Maxt) + `,`, + `Ref:` + fmt.Sprintf("%v", this.Ref) + `,`, + `}`, + }, "") + return s +} +func valueToStringTypes(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *Chunk) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Chunk: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Chunk: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= Chunk_Encoding(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...) + if m.Data == nil { + m.Data = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + m.Hash = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Hash |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Series) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Series: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Series: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Chunks = append(m.Chunks, AggrChunk{}) + if err := m.Chunks[len(m.Chunks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AggrChunk) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AggrChunk: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AggrChunk: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MinTime", wireType) + } + m.MinTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MinTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxTime", wireType) + } + m.MaxTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Raw", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Raw == nil { + m.Raw = &Chunk{} + } + if err := m.Raw.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Count == nil { + m.Count = &Chunk{} + } + if err := m.Count.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Sum == nil { + m.Sum = &Chunk{} + } + if err := m.Sum.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Min", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Min == nil { + m.Min = &Chunk{} + } + if err := m.Min.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Max", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Max == nil { + m.Max = &Chunk{} + } + if err := m.Max.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Counter", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Counter == nil { + m.Counter = &Chunk{} + } + if err := m.Counter.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LabelMatcher) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LabelMatcher: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LabelMatcher: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= LabelMatcher_Type(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SelectedSeries) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SelectedSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SelectedSeries: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Chunks = append(m.Chunks, &ChunkMeta{}) + if err := m.Chunks[len(m.Chunks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChunkMeta) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ChunkMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChunkMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Mint", wireType) + } + m.Mint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Mint |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Maxt", wireType) + } + m.Maxt = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Maxt |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Ref", wireType) + } + m.Ref = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Ref |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipTypes(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/storegateway/typespb/types.proto b/pkg/storegateway/typespb/types.proto new file mode 100644 index 0000000000..1e38e06362 --- /dev/null +++ b/pkg/storegateway/typespb/types.proto @@ -0,0 +1,88 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +syntax = "proto3"; +package typespb; + +option go_package = "typespb"; + +import "gogoproto/gogo.proto"; +import "github.com/thanos-io/thanos/pkg/store/labelpb/types.proto"; + +option (gogoproto.sizer_all) = true; +option (gogoproto.marshaler_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +// Do not generate XXX fields to reduce memory footprint and opening a door +// for zero-copy casts to/from prometheus data types. +option (gogoproto.goproto_unkeyed_all) = false; +option (gogoproto.goproto_unrecognized_all) = false; +option (gogoproto.goproto_sizecache_all) = false; + +message Chunk { + enum Encoding { + XOR = 0; + HISTOGRAM = 1; + FLOAT_HISTOGRAM = 2; + } + Encoding type = 1; + bytes data = 2; + uint64 hash = 3 [(gogoproto.nullable) = true]; +} + +message Series { + repeated thanos.Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; + repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; +} + +message AggrChunk { + int64 min_time = 1; + int64 max_time = 2; + + Chunk raw = 3; + Chunk count = 4; + Chunk sum = 5; + Chunk min = 6; + Chunk max = 7; + Chunk counter = 8; +} + +// Matcher specifies a rule, which can match or set of labels or not. +message LabelMatcher { + enum Type { + EQ = 0; // = + NEQ = 1; // != + RE = 2; // =~ + NRE = 3; // !~ + } + Type type = 1; + string name = 2; + string value = 3; +} + +/// PartialResponseStrategy controls partial response handling. +enum PartialResponseStrategy { + /// WARN strategy tells server to treat any error that will related to single StoreAPI (e.g missing chunk series because of underlying + /// storeAPI is temporarily not available) as warning which will not fail the whole query (still OK response). + /// Server should produce those as a warnings field in response. + WARN = 0; + /// ABORT strategy tells server to treat any error that will related to single StoreAPI (e.g missing chunk series because of underlying + /// storeAPI is temporarily not available) as the gRPC error that aborts the query. + /// + /// This is especially useful for any rule/alert evaluations on top of StoreAPI which usually does not tolerate partial + /// errors. + ABORT = 1; +} + +message SelectedSeries { + string blockId = 1; + repeated thanos.Label labels = 2 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; + repeated ChunkMeta chunks = 3; +} + +message ChunkMeta { + int64 mint = 1; + int64 maxt = 2; + uint64 ref = 3; +}