diff --git a/cmd/thanos/query.go b/cmd/thanos/query.go index 3f76d73fd3..3ae75cffe8 100644 --- a/cmd/thanos/query.go +++ b/cmd/thanos/query.go @@ -111,6 +111,9 @@ func registerQuery(app *extkingpin.App) { maxConcurrentSelects := cmd.Flag("query.max-concurrent-select", "Maximum number of select requests made concurrently per a query."). Default("4").Int() + maxConcurrentDecompressWorkers := cmd.Flag("query.max-concurrent-decompress-workers", "Maximum number of workers spawned to decompress a set of compressed storepb.Series. Setting this to higher than zero enables label compression during querying - CPU usage will be slightly higher whilst network usage will be significantly lower."). + Default("0").Int() + queryConnMetricLabels := cmd.Flag("query.conn-metric.label", "Optional selection of query connection metric labels to be collected from endpoint set"). Default(string(query.ExternalLabels), string(query.StoreType)). Enums(string(query.ExternalLabels), string(query.StoreType)) @@ -321,6 +324,7 @@ func registerQuery(app *extkingpin.App) { *queryTelemetrySamplesQuantiles, *queryTelemetrySeriesQuantiles, promqlEngineType(*promqlEngine), + *maxConcurrentDecompressWorkers, ) }) } @@ -397,6 +401,7 @@ func runQuery( queryTelemetrySamplesQuantiles []int64, queryTelemetrySeriesQuantiles []int64, promqlEngine promqlEngineType, + maxConcurrentDecompressWorkers int, ) error { if alertQueryURL == "" { lastColon := strings.LastIndex(httpBindAddr, ":") @@ -507,7 +512,16 @@ func runQuery( endpointInfoTimeout, queryConnMetricLabels..., ) - proxy = store.NewProxyStore(logger, reg, endpoints.GetStoreClients, component.Query, selectorLset, storeResponseTimeout, store.RetrievalStrategy(grpcProxyStrategy)) + proxy = store.NewProxyStore( + logger, + reg, + endpoints.GetStoreClients, + component.Query, + selectorLset, + storeResponseTimeout, + store.RetrievalStrategy(grpcProxyStrategy), + maxConcurrentDecompressWorkers > 0, + ) rulesProxy = rules.NewProxy(logger, endpoints.GetRulesClients) targetsProxy = targets.NewProxy(logger, endpoints.GetTargetsClients) metadataProxy = metadata.NewProxy(logger, endpoints.GetMetricMetadataClients) @@ -518,6 +532,7 @@ func runQuery( proxy, maxConcurrentSelects, queryTimeout, + maxConcurrentDecompressWorkers, ) ) diff --git a/cmd/thanos/receive.go b/cmd/thanos/receive.go index 85c1bd64e6..77ae407665 100644 --- a/cmd/thanos/receive.go +++ b/cmd/thanos/receive.go @@ -313,6 +313,7 @@ func runReceive( labels.Labels{}, 0, store.LazyRetrieval, + false, ) rw := store.ReadWriteTSDBStore{ StoreServer: mts, diff --git a/docs/components/query.md b/docs/components/query.md index 9de81aab5a..d1e7db69f6 100644 --- a/docs/components/query.md +++ b/docs/components/query.md @@ -373,6 +373,13 @@ Flags: If unset it will use the promql default of 5m. --query.max-concurrent=20 Maximum number of queries processed concurrently by query node. + --query.max-concurrent-decompress-workers=0 + Maximum number of workers spawned to + decompress a set of compressed storepb.Series. + Setting this to higher than zero enables label + compression during querying - CPU usage will + be slightly higher whilst network usage will be + significantly lower. --query.max-concurrent-select=4 Maximum number of select requests made concurrently per a query. diff --git a/pkg/api/query/v1_test.go b/pkg/api/query/v1_test.go index 55706a1bc9..69fa930d5f 100644 --- a/pkg/api/query/v1_test.go +++ b/pkg/api/query/v1_test.go @@ -192,7 +192,7 @@ func TestQueryEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout, 0), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), @@ -733,7 +733,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout, 0), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), @@ -746,7 +746,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout, 0), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), diff --git a/pkg/query/querier.go b/pkg/query/querier.go index e0eeab489f..e2a9f16dfa 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -5,15 +5,19 @@ package query import ( "context" + "fmt" "sort" "strings" "sync" "time" "github.com/go-kit/log" + "github.com/gogo/protobuf/types" "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" @@ -22,6 +26,7 @@ import ( "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/gate" "github.com/thanos-io/thanos/pkg/store" + "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/thanos-io/thanos/pkg/tracing" @@ -65,6 +70,7 @@ func NewQueryableCreator( proxy storepb.StoreServer, maxConcurrentSelects int, selectTimeout time.Duration, + maxConcurrentDecompressWorkers int, ) QueryableCreator { return func( deduplicate bool, @@ -90,54 +96,57 @@ func NewQueryableCreator( gateProviderFn: func() gate.Gate { return gate.New(reg, maxConcurrentSelects) }, - maxConcurrentSelects: maxConcurrentSelects, - selectTimeout: selectTimeout, - enableQueryPushdown: enableQueryPushdown, - shardInfo: shardInfo, - seriesStatsReporter: seriesStatsReporter, + maxConcurrentSelects: maxConcurrentSelects, + selectTimeout: selectTimeout, + enableQueryPushdown: enableQueryPushdown, + shardInfo: shardInfo, + seriesStatsReporter: seriesStatsReporter, + maxConcurrentDecompressWorkers: maxConcurrentDecompressWorkers, } } } type queryable struct { - logger log.Logger - replicaLabels []string - storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer - deduplicate bool - maxResolutionMillis int64 - partialResponse bool - skipChunks bool - gateProviderFn func() gate.Gate - maxConcurrentSelects int - selectTimeout time.Duration - enableQueryPushdown bool - shardInfo *storepb.ShardInfo - seriesStatsReporter seriesStatsReporter + logger log.Logger + replicaLabels []string + storeDebugMatchers [][]*labels.Matcher + proxy storepb.StoreServer + deduplicate bool + maxResolutionMillis int64 + partialResponse bool + skipChunks bool + gateProviderFn func() gate.Gate + maxConcurrentSelects int + selectTimeout time.Duration + enableQueryPushdown bool + shardInfo *storepb.ShardInfo + seriesStatsReporter seriesStatsReporter + maxConcurrentDecompressWorkers int } // Querier returns a new storage querier against the underlying proxy store API. func (q *queryable) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { - return newQuerier(ctx, q.logger, mint, maxt, q.replicaLabels, q.storeDebugMatchers, q.proxy, q.deduplicate, q.maxResolutionMillis, q.partialResponse, q.enableQueryPushdown, q.skipChunks, q.gateProviderFn(), q.selectTimeout, q.shardInfo, q.seriesStatsReporter), nil + return newQuerier(ctx, q.logger, mint, maxt, q.replicaLabels, q.storeDebugMatchers, q.proxy, q.deduplicate, q.maxResolutionMillis, q.partialResponse, q.enableQueryPushdown, q.skipChunks, q.gateProviderFn(), q.selectTimeout, q.shardInfo, q.seriesStatsReporter, q.maxConcurrentDecompressWorkers), nil } type querier struct { - ctx context.Context - logger log.Logger - cancel func() - mint, maxt int64 - replicaLabels map[string]struct{} - storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer - deduplicate bool - maxResolutionMillis int64 - partialResponse bool - enableQueryPushdown bool - skipChunks bool - selectGate gate.Gate - selectTimeout time.Duration - shardInfo *storepb.ShardInfo - seriesStatsReporter seriesStatsReporter + ctx context.Context + logger log.Logger + cancel func() + mint, maxt int64 + replicaLabels map[string]struct{} + storeDebugMatchers [][]*labels.Matcher + proxy storepb.StoreServer + deduplicate bool + maxResolutionMillis int64 + partialResponse bool + enableQueryPushdown bool + skipChunks bool + selectGate gate.Gate + selectTimeout time.Duration + shardInfo *storepb.ShardInfo + seriesStatsReporter seriesStatsReporter + maxConcurrentDecompressWorkers int } // newQuerier creates implementation of storage.Querier that fetches data from the proxy @@ -159,6 +168,7 @@ func newQuerier( selectTimeout time.Duration, shardInfo *storepb.ShardInfo, seriesStatsReporter seriesStatsReporter, + maxConcurrentDecompressWorkers int, ) *querier { if logger == nil { logger = log.NewNopLogger() @@ -176,18 +186,19 @@ func newQuerier( selectGate: selectGate, selectTimeout: selectTimeout, - mint: mint, - maxt: maxt, - replicaLabels: rl, - storeDebugMatchers: storeDebugMatchers, - proxy: proxy, - deduplicate: deduplicate, - maxResolutionMillis: maxResolutionMillis, - partialResponse: partialResponse, - skipChunks: skipChunks, - enableQueryPushdown: enableQueryPushdown, - shardInfo: shardInfo, - seriesStatsReporter: seriesStatsReporter, + mint: mint, + maxt: maxt, + replicaLabels: rl, + storeDebugMatchers: storeDebugMatchers, + proxy: proxy, + deduplicate: deduplicate, + maxResolutionMillis: maxResolutionMillis, + partialResponse: partialResponse, + skipChunks: skipChunks, + enableQueryPushdown: enableQueryPushdown, + shardInfo: shardInfo, + seriesStatsReporter: seriesStatsReporter, + maxConcurrentDecompressWorkers: maxConcurrentDecompressWorkers, } } @@ -203,6 +214,121 @@ type seriesServer struct { seriesSet []storepb.Series seriesSetStats storepb.SeriesStatsCounter warnings []string + symbolTables []map[uint64]string + + compressedSeriesSet []storepb.CompressedSeries +} + +func (s *seriesServer) decompressSeriesIndex(i int) (*storepb.Series, error) { + newSeries := &storepb.Series{ + Chunks: s.compressedSeriesSet[i].Chunks, + } + + lbls := make(labels.Labels, 0, len(s.compressedSeriesSet[i].Labels)) + + for _, cLabel := range s.compressedSeriesSet[i].Labels { + var name, val string + for _, symTable := range s.symbolTables { + if foundName, ok := symTable[uint64(cLabel.NameRef)]; ok { + name = foundName + } + + if foundValue, ok := symTable[uint64(cLabel.ValueRef)]; ok { + val = foundValue + } + + if name != "" && val != "" { + break + } + } + if name == "" || val == "" { + return nil, fmt.Errorf("series %+v references do not exist", cLabel) + } + + lbls = append(lbls, labels.Label{ + Name: name, + Value: val, + }) + } + + newSeries.Labels = labelpb.ZLabelsFromPromLabels(lbls) + return newSeries, nil +} + +func (s *seriesServer) DecompressSeries(maxWorkers int) error { + if len(s.compressedSeriesSet) == 0 { + return nil + } + + workerInput := make(chan int) + workerOutput := make(chan *storepb.Series) + + var elements uint64 + for _, css := range s.compressedSeriesSet { + elements += uint64(len(css.Labels)) * 2 + } + + newSeriesSet := make([]storepb.Series, 0, len(s.seriesSet)+len(s.compressedSeriesSet)) + newSeriesSet = append(newSeriesSet, s.seriesSet...) + + // NOTE(GiedriusS): Ballpark estimate. With more workers I got slower results. + workerCount := 1 + (elements / 2000000) + if workerCount == 1 || maxWorkers < 1 { + for i := range s.compressedSeriesSet { + decompressedSeries, err := s.decompressSeriesIndex(i) + if err != nil { + return fmt.Errorf("decompressing element %d: %w", i, err) + } + + newSeriesSet = append(newSeriesSet, *decompressedSeries) + } + s.seriesSet = newSeriesSet + return nil + } + + if maxWorkers > 0 && workerCount > uint64(maxWorkers) { + workerCount = uint64(maxWorkers) + } + + wg := &sync.WaitGroup{} + errLock := sync.Mutex{} + errs := tsdb_errors.NewMulti() + + for i := uint64(0); i < workerCount; i++ { + wg.Add(1) + + go func() { + defer wg.Done() + for ind := range workerInput { + decompressedSeries, err := s.decompressSeriesIndex(ind) + if err != nil { + errLock.Lock() + errs.Add(fmt.Errorf("decompressing element %d: %w", ind, err)) + errLock.Unlock() + return + } + + workerOutput <- decompressedSeries + } + }() + } + + go func() { + for i := range s.compressedSeriesSet { + workerInput <- i + } + + close(workerInput) + wg.Wait() + close(workerOutput) + }() + + for wo := range workerOutput { + newSeriesSet = append(newSeriesSet, *wo) + } + s.seriesSet = newSeriesSet + + return errs.Err() } func (s *seriesServer) Send(r *storepb.SeriesResponse) error { @@ -217,6 +343,24 @@ func (s *seriesServer) Send(r *storepb.SeriesResponse) error { return nil } + if r.GetCompressedSeries() != nil { + s.compressedSeriesSet = append(s.compressedSeriesSet, *r.GetCompressedSeries()) + s.seriesSetStats.Count(r.GetCompressedSeries()) + return nil + } + + if r.GetHints() != nil { + var seriesResponseHints hintspb.SeriesResponseHints + + // Some other, unknown type. Skip it. + if err := types.UnmarshalAny(r.GetHints(), &seriesResponseHints); err != nil { + return nil + } + + s.symbolTables = append(s.symbolTables, seriesResponseHints.StringSymbolTable) + return nil + } + // Unsupported field, skip. return nil } @@ -364,6 +508,10 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . warns = append(warns, errors.New(w)) } + if err := resp.DecompressSeries(q.maxConcurrentDecompressWorkers); err != nil { + return nil, storepb.SeriesStatsCounter{}, errors.Wrap(err, "decompressing series") + } + // Delete the metric's name from the result because that's what the // PromQL does either way and we want our iterator to work with data // that was either pushed down or not. diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 2789d72cce..959df3debe 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -42,7 +42,7 @@ type sample struct { func TestQueryableCreator_MaxResolution(t *testing.T) { testProxy := &testStoreServer{resps: []*storepb.SeriesResponse{}} - queryableCreator := NewQueryableCreator(nil, nil, testProxy, 2, 5*time.Second) + queryableCreator := NewQueryableCreator(nil, nil, testProxy, 2, 5*time.Second, 0) oneHourMillis := int64(1*time.Hour) / int64(time.Millisecond) queryable := queryableCreator( @@ -88,6 +88,7 @@ func TestQuerier_DownsampledData(t *testing.T) { testProxy, 2, timeout, + 0, )(false, nil, nil, @@ -391,7 +392,7 @@ func TestQuerier_Select_AfterPromQL(t *testing.T) { g := gate.New(2) mq := &mockedQueryable{ Creator: func(mint, maxt int64) storage.Querier { - return newQuerier(context.Background(), nil, mint, maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + return newQuerier(context.Background(), nil, mint, maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter, 0) }, } t.Cleanup(func() { @@ -635,7 +636,7 @@ func TestQuerier_Select(t *testing.T) { {dedup: true, expected: []series{tcase.expectedAfterDedup}}, } { g := gate.New(2) - q := newQuerier(context.Background(), nil, tcase.mint, tcase.maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, func(i storepb.SeriesStatsCounter) {}) + q := newQuerier(context.Background(), nil, tcase.mint, tcase.maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, func(i storepb.SeriesStatsCounter) {}, 0) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) t.Run(fmt.Sprintf("dedup=%v", sc.dedup), func(t *testing.T) { @@ -887,7 +888,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 100 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter, 0) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) @@ -957,7 +958,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 5 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter, 0) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) diff --git a/pkg/query/query_test.go b/pkg/query/query_test.go index 6c68fc5e1a..822aedfa4d 100644 --- a/pkg/query/query_test.go +++ b/pkg/query/query_test.go @@ -37,9 +37,10 @@ func TestQuerier_Proxy(t *testing.T) { logger, nil, store.NewProxyStore(logger, nil, func() []store.Client { return clients }, - component.Debug, nil, 5*time.Minute, store.EagerRetrieval), + component.Debug, nil, 5*time.Minute, store.EagerRetrieval, false), 1000000, 5*time.Minute, + 0, ) createQueryableFn := func(stores []*testStore) storage.Queryable { diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 23517b8d91..549ecf2d68 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -350,7 +350,7 @@ type BucketStore struct { postingOffsetsInMemSampling int // Enables hints in the Series() response. - enableSeriesResponseHints bool + enableQueriedBlocksHints bool enableChunkHashCalculation bool } @@ -477,7 +477,7 @@ func NewBucketStore( partitioner: partitioner, enableCompatibilityLabel: enableCompatibilityLabel, postingOffsetsInMemSampling: postingOffsetsInMemSampling, - enableSeriesResponseHints: enableSeriesResponseHints, + enableQueriedBlocksHints: enableSeriesResponseHints, enableChunkHashCalculation: enableChunkHashCalculation, seriesBatchSize: SeriesBatchSize, } @@ -1204,6 +1204,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } s.mtx.RLock() + for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) if !ok { @@ -1220,7 +1221,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie blk := b gctx := gctx - if s.enableSeriesResponseHints { + if s.enableQueriedBlocksHints { // Keep track of queried blocks. resHints.AddQueriedBlock(blk.meta.ULID) } @@ -1271,6 +1272,8 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie shardMatcher, false, s.metrics.emptyPostingCount, + req.MaximumStringSlots, + nil, ) mtx.Lock() @@ -1332,6 +1335,8 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie s.metrics.seriesBlocksQueried.Observe(float64(stats.blocksQueried)) } + symbolTableBuilder := newSymbolTableBuilder(req.MaximumStringSlots) + // Merge the sub-results from each selected block. tracing.DoInSpan(ctx, "bucket_store_merge_all", func(ctx context.Context) { defer func() { @@ -1359,7 +1364,35 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie stats.mergedChunksCount += len(series.Chunks) s.metrics.chunkSizeBytes.Observe(float64(chunksSize(series.Chunks))) } + + lset := series.Labels + + var compressedResponse bool = true + compressedLabels := make([]labelpb.CompressedLabel, 0, len(lset)) + + for _, lbl := range lset { + nameRef, nok := symbolTableBuilder.getOrStoreString(lbl.Name) + valueRef, vok := symbolTableBuilder.getOrStoreString(lbl.Value) + + if !nok || !vok { + compressedResponse = false + break + } else if compressedResponse && nok && vok { + compressedLabels = append(compressedLabels, labelpb.CompressedLabel{ + NameRef: nameRef, + ValueRef: valueRef, + }) + } + } + + if compressedResponse { + at = storepb.NewCompressedSeriesResponse(&storepb.CompressedSeries{ + Labels: compressedLabels, + Chunks: series.Chunks, + }) + } } + if err = srv.Send(at); err != nil { err = status.Error(codes.Unknown, errors.Wrap(err, "send series response").Error()) return @@ -1374,7 +1407,9 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie return err } - if s.enableSeriesResponseHints { + resHints.StringSymbolTable = symbolTableBuilder.getTable() + + { var anyHints *types.Any if anyHints, err = types.MarshalAny(resHints); err != nil { diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index c115be4d6b..6e72dc89ec 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -1379,6 +1379,9 @@ func benchBucketSeries(t testutil.TB, skipChunk bool, samplesPerSeries, totalSer }, // This does not cut chunks properly, but those are assured against for non benchmarks only, where we use 100% case only. ExpectedSeries: series[:seriesCut], + ExpectedHints: []hintspb.SeriesResponseHints{ + {}, + }, }) } storetestutil.TestServerSeries(t, st, bCases...) diff --git a/pkg/store/hintspb/hints.pb.go b/pkg/store/hintspb/hints.pb.go index c22d52586e..2e5f0d7096 100644 --- a/pkg/store/hintspb/hints.pb.go +++ b/pkg/store/hintspb/hints.pb.go @@ -68,6 +68,8 @@ 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"` + // Symbol table for label names/values. + StringSymbolTable map[uint64]string `protobuf:"bytes,2,rep,name=string_symbol_table,json=stringSymbolTable,proto3" json:"string_symbol_table" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (m *SeriesResponseHints) Reset() { *m = SeriesResponseHints{} } @@ -299,6 +301,7 @@ var xxx_messageInfo_LabelValuesResponseHints proto.InternalMessageInfo func init() { proto.RegisterType((*SeriesRequestHints)(nil), "hintspb.SeriesRequestHints") proto.RegisterType((*SeriesResponseHints)(nil), "hintspb.SeriesResponseHints") + proto.RegisterMapType((map[uint64]string)(nil), "hintspb.SeriesResponseHints.StringSymbolTableEntry") proto.RegisterType((*Block)(nil), "hintspb.Block") proto.RegisterType((*LabelNamesRequestHints)(nil), "hintspb.LabelNamesRequestHints") proto.RegisterType((*LabelNamesResponseHints)(nil), "hintspb.LabelNamesResponseHints") @@ -309,26 +312,31 @@ func init() { func init() { proto.RegisterFile("store/hintspb/hints.proto", fileDescriptor_b82aa23c4c11e83f) } var fileDescriptor_b82aa23c4c11e83f = []byte{ - // 295 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2c, 0x2e, 0xc9, 0x2f, - 0x4a, 0xd5, 0xcf, 0xc8, 0xcc, 0x2b, 0x29, 0x2e, 0x48, 0x82, 0xd0, 0x7a, 0x05, 0x45, 0xf9, 0x25, - 0xf9, 0x42, 0xec, 0x50, 0x41, 0x29, 0x91, 0xf4, 0xfc, 0xf4, 0x7c, 0xb0, 0x98, 0x3e, 0x88, 0x05, - 0x91, 0x96, 0x82, 0xea, 0x04, 0x93, 0x05, 0x49, 0xfa, 0x25, 0x95, 0x05, 0xa9, 0x50, 0x9d, 0x4a, - 0xe1, 0x5c, 0x42, 0xc1, 0xa9, 0x45, 0x99, 0xa9, 0xc5, 0x41, 0xa9, 0x85, 0xa5, 0xa9, 0xc5, 0x25, - 0x1e, 0x20, 0x83, 0x84, 0x1c, 0xb9, 0xf8, 0x92, 0x72, 0xf2, 0x93, 0xb3, 0xe3, 0x73, 0x13, 0x4b, - 0x92, 0x33, 0x52, 0x8b, 0x8a, 0x25, 0x18, 0x15, 0x98, 0x35, 0xb8, 0x8d, 0x44, 0xf4, 0x4a, 0x32, - 0x12, 0xf3, 0xf2, 0x8b, 0xf5, 0x7c, 0x12, 0x93, 0x52, 0x73, 0x7c, 0x21, 0x92, 0x4e, 0x2c, 0x27, - 0xee, 0xc9, 0x33, 0x04, 0xf1, 0x82, 0x75, 0x40, 0xc5, 0x8a, 0x95, 0x82, 0xb8, 0x84, 0x61, 0x06, - 0x17, 0x17, 0xe4, 0xe7, 0x15, 0xa7, 0x42, 0x4c, 0xb6, 0xe6, 0xe2, 0x2b, 0x2c, 0x05, 0x89, 0xa7, - 0xc4, 0x83, 0xd5, 0xc3, 0x4c, 0xe6, 0xd3, 0x83, 0x7a, 0x41, 0xcf, 0x09, 0x24, 0x0c, 0x33, 0x13, - 0xaa, 0x16, 0x2c, 0x56, 0xac, 0x24, 0xce, 0xc5, 0x0a, 0x66, 0x09, 0xf1, 0x71, 0x31, 0x65, 0xa6, - 0x48, 0x30, 0x2a, 0x30, 0x6a, 0x70, 0x06, 0x31, 0x65, 0xa6, 0x28, 0x45, 0x73, 0x89, 0x81, 0x5d, - 0xe4, 0x97, 0x98, 0x4b, 0x7d, 0x9f, 0x84, 0x71, 0x89, 0x23, 0x1b, 0x4e, 0x35, 0xdf, 0xc4, 0x40, - 0xcd, 0x0d, 0x4b, 0xcc, 0x29, 0xa5, 0xbe, 0xab, 0xc3, 0xb9, 0x24, 0x50, 0x4c, 0xa7, 0x96, 0xb3, - 0x9d, 0x54, 0x4f, 0x3c, 0x94, 0x63, 0x38, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x07, - 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, 0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, 0x39, 0x86, - 0x28, 0x58, 0x4a, 0x4c, 0x62, 0x03, 0xa7, 0x2f, 0x63, 0x40, 0x00, 0x00, 0x00, 0xff, 0xff, 0x47, - 0x2f, 0x08, 0x1f, 0xb6, 0x02, 0x00, 0x00, + // 382 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x93, 0xc1, 0x4a, 0xc3, 0x30, + 0x18, 0xc7, 0x9b, 0x6e, 0x53, 0x16, 0xb1, 0x68, 0x37, 0xb6, 0xba, 0x43, 0x1d, 0x05, 0x61, 0xa7, + 0x0e, 0xdc, 0x45, 0xf4, 0xe4, 0x50, 0xf0, 0xa0, 0x1e, 0x3a, 0xd9, 0x40, 0x85, 0xd2, 0x6c, 0x61, + 0x2b, 0xeb, 0x9a, 0xae, 0xc9, 0x84, 0x9e, 0x7d, 0x01, 0x1f, 0x6b, 0xc7, 0x1d, 0x3d, 0x89, 0x6e, + 0x2f, 0x22, 0x49, 0x53, 0x51, 0xd9, 0xb1, 0x97, 0x36, 0xfd, 0x7f, 0xfd, 0xfd, 0xf2, 0xe5, 0x83, + 0xc0, 0x23, 0xca, 0x48, 0x8c, 0xdb, 0x13, 0x3f, 0x64, 0x34, 0x42, 0xe9, 0xdb, 0x8e, 0x62, 0xc2, + 0x88, 0xbe, 0x2b, 0xc3, 0x46, 0x75, 0x4c, 0xc6, 0x44, 0x64, 0x6d, 0xbe, 0x4a, 0xcb, 0x0d, 0x49, + 0x8a, 0x67, 0x84, 0xda, 0x2c, 0x89, 0xb0, 0x24, 0xad, 0x01, 0xd4, 0x7b, 0x38, 0xf6, 0x31, 0x75, + 0xf0, 0x7c, 0x81, 0x29, 0xbb, 0xe1, 0x22, 0xfd, 0x12, 0x6a, 0x28, 0x20, 0xc3, 0xa9, 0x3b, 0xf3, + 0xd8, 0x70, 0x82, 0x63, 0x6a, 0x80, 0x66, 0xa1, 0xb5, 0x77, 0x5a, 0xb5, 0xd9, 0xc4, 0x0b, 0x09, + 0xb5, 0x6f, 0x3d, 0x84, 0x83, 0xbb, 0xb4, 0xd8, 0x2d, 0x2e, 0x3f, 0x8e, 0x15, 0x67, 0x5f, 0x10, + 0x32, 0xa3, 0xd6, 0xab, 0x0a, 0x2b, 0x99, 0x99, 0x46, 0x24, 0xa4, 0x38, 0x55, 0x5f, 0x40, 0x6d, + 0xbe, 0xe0, 0xf9, 0xc8, 0x15, 0x40, 0xa6, 0xd6, 0x6c, 0x79, 0x06, 0xbb, 0xcb, 0xe3, 0x4c, 0x2a, + 0xff, 0x15, 0x19, 0xd5, 0x7d, 0x58, 0xa1, 0x2c, 0xf6, 0xc3, 0xb1, 0x4b, 0x93, 0x19, 0x22, 0x81, + 0xcb, 0x3c, 0x14, 0x60, 0x43, 0x15, 0x86, 0xce, 0x8f, 0x61, 0xcb, 0xbe, 0x76, 0x4f, 0x70, 0x3d, + 0x81, 0x3d, 0x70, 0xea, 0x3a, 0x64, 0x71, 0x22, 0xb7, 0x39, 0xa4, 0xff, 0xab, 0x8d, 0x2b, 0x58, + 0xdb, 0x8e, 0xe8, 0x07, 0xb0, 0x30, 0xc5, 0x89, 0x01, 0x9a, 0xa0, 0x55, 0x74, 0xf8, 0x52, 0xaf, + 0xc2, 0xd2, 0x8b, 0x17, 0x2c, 0x78, 0x23, 0xa0, 0x55, 0x76, 0xd2, 0x8f, 0x73, 0xf5, 0x0c, 0x58, + 0x75, 0x58, 0x12, 0xad, 0xeb, 0x1a, 0x54, 0xfd, 0x91, 0x60, 0xca, 0x8e, 0xea, 0x8f, 0xac, 0x27, + 0x58, 0x13, 0x33, 0xbc, 0xf7, 0x66, 0xf9, 0xcf, 0xbe, 0x0f, 0xeb, 0xbf, 0xe5, 0x79, 0x8d, 0xdf, + 0x7a, 0x96, 0xde, 0x3e, 0x3f, 0x5f, 0xee, 0x5d, 0x0f, 0xa0, 0xf1, 0xc7, 0x9e, 0x57, 0xdb, 0xdd, + 0x93, 0xe5, 0x97, 0xa9, 0x2c, 0xd7, 0x26, 0x58, 0xad, 0x4d, 0xf0, 0xb9, 0x36, 0xc1, 0xdb, 0xc6, + 0x54, 0x56, 0x1b, 0x53, 0x79, 0xdf, 0x98, 0xca, 0x63, 0x76, 0x77, 0xd0, 0x8e, 0xb8, 0x11, 0x9d, + 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x27, 0x33, 0xcb, 0xf7, 0x68, 0x03, 0x00, 0x00, } func (m *SeriesRequestHints) Marshal() (dAtA []byte, err error) { @@ -388,6 +396,23 @@ func (m *SeriesResponseHints) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StringSymbolTable) > 0 { + for k := range m.StringSymbolTable { + v := m.StringSymbolTable[k] + baseI := i + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintHints(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + i = encodeVarintHints(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintHints(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x12 + } + } if len(m.QueriedBlocks) > 0 { for iNdEx := len(m.QueriedBlocks) - 1; iNdEx >= 0; iNdEx-- { { @@ -621,6 +646,14 @@ func (m *SeriesResponseHints) Size() (n int) { n += 1 + l + sovHints(uint64(l)) } } + if len(m.StringSymbolTable) > 0 { + for k, v := range m.StringSymbolTable { + _ = k + _ = v + mapEntrySize := 1 + sovHints(uint64(k)) + 1 + len(v) + sovHints(uint64(len(v))) + n += mapEntrySize + 1 + sovHints(uint64(mapEntrySize)) + } + } return n } @@ -850,6 +883,119 @@ func (m *SeriesResponseHints) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StringSymbolTable", 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.StringSymbolTable == nil { + m.StringSymbolTable = make(map[uint64]string) + } + var mapkey uint64 + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHints + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLengthHints + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return ErrInvalidLengthHints + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := skipHints(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthHints + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.StringSymbolTable[mapkey] = mapvalue + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipHints(dAtA[iNdEx:]) diff --git a/pkg/store/hintspb/hints.proto b/pkg/store/hintspb/hints.proto index f7cf68d3ff..33e27a3759 100644 --- a/pkg/store/hintspb/hints.proto +++ b/pkg/store/hintspb/hints.proto @@ -29,6 +29,9 @@ message SeriesRequestHints { message SeriesResponseHints { /// queried_blocks is the list of blocks that have been queried. repeated Block queried_blocks = 1 [(gogoproto.nullable) = false]; + + // Symbol table for label names/values. + map string_symbol_table = 2 [(gogoproto.nullable) = false]; } message Block { @@ -58,4 +61,4 @@ message LabelValuesRequestHints { message LabelValuesResponseHints { /// queried_blocks is the list of blocks that have been queried. repeated Block queried_blocks = 1 [(gogoproto.nullable) = false]; -} \ No newline at end of file +} diff --git a/pkg/store/labelpb/label.go b/pkg/store/labelpb/label.go index bc8b83f59b..158a7f534b 100644 --- a/pkg/store/labelpb/label.go +++ b/pkg/store/labelpb/label.go @@ -7,6 +7,7 @@ package labelpb import ( + "encoding/binary" "encoding/json" "fmt" "io" @@ -356,6 +357,38 @@ func DeepCopy(lbls []ZLabel) []ZLabel { return ret } +// HashCompressedWithPrefix returns a hash for the given prefix and labels. +func HashCompressedWithPrefix(prefix string, lbls []CompressedLabel) uint64 { + // Use xxhash.Sum64(b) for fast path as it's faster. + b := make([]byte, 0, 1024) + b = append(b, prefix...) + b = append(b, sep[0]) + + for i, v := range lbls { + if len(b)+8+8+2 >= cap(b) { + // If labels entry is 1KB allocate do not allocate whole entry. + h := xxhash.New() + _, _ = h.Write(b) + + refStorage := make([]byte, 8) + for _, v := range lbls[i:] { + binary.LittleEndian.PutUint64(refStorage, v.NameRef) + _, _ = h.Write(refStorage) + _, _ = h.Write(sep) + binary.LittleEndian.PutUint64(refStorage, v.ValueRef) + _, _ = h.Write(refStorage) + _, _ = h.Write(sep) + } + return h.Sum64() + } + b = binary.LittleEndian.AppendUint64(b, v.NameRef) + b = append(b, sep[0]) + b = binary.LittleEndian.AppendUint64(b, v.ValueRef) + b = append(b, sep[0]) + } + return xxhash.Sum64(b) +} + // HashWithPrefix returns a hash for the given prefix and labels. func HashWithPrefix(prefix string, lbls []ZLabel) uint64 { // Use xxhash.Sum64(b) for fast path as it's faster. diff --git a/pkg/store/labelpb/types.pb.go b/pkg/store/labelpb/types.pb.go index 3dd6d97299..40aeff582e 100644 --- a/pkg/store/labelpb/types.pb.go +++ b/pkg/store/labelpb/types.pb.go @@ -62,6 +62,44 @@ func (m *Label) XXX_DiscardUnknown() { var xxx_messageInfo_Label proto.InternalMessageInfo +type CompressedLabel struct { + NameRef uint64 `protobuf:"varint,1,opt,name=name_ref,json=nameRef,proto3" json:"name_ref,omitempty"` + ValueRef uint64 `protobuf:"varint,2,opt,name=value_ref,json=valueRef,proto3" json:"value_ref,omitempty"` +} + +func (m *CompressedLabel) Reset() { *m = CompressedLabel{} } +func (m *CompressedLabel) String() string { return proto.CompactTextString(m) } +func (*CompressedLabel) ProtoMessage() {} +func (*CompressedLabel) Descriptor() ([]byte, []int) { + return fileDescriptor_cdcc9e7dae4870e8, []int{1} +} +func (m *CompressedLabel) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CompressedLabel) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CompressedLabel.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 *CompressedLabel) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompressedLabel.Merge(m, src) +} +func (m *CompressedLabel) XXX_Size() int { + return m.Size() +} +func (m *CompressedLabel) XXX_DiscardUnknown() { + xxx_messageInfo_CompressedLabel.DiscardUnknown(m) +} + +var xxx_messageInfo_CompressedLabel proto.InternalMessageInfo + type LabelSet struct { Labels []Label `protobuf:"bytes,1,rep,name=labels,proto3" json:"labels"` } @@ -70,7 +108,7 @@ func (m *LabelSet) Reset() { *m = LabelSet{} } func (m *LabelSet) String() string { return proto.CompactTextString(m) } func (*LabelSet) ProtoMessage() {} func (*LabelSet) Descriptor() ([]byte, []int) { - return fileDescriptor_cdcc9e7dae4870e8, []int{1} + return fileDescriptor_cdcc9e7dae4870e8, []int{2} } func (m *LabelSet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -107,7 +145,7 @@ func (m *ZLabelSet) Reset() { *m = ZLabelSet{} } func (m *ZLabelSet) String() string { return proto.CompactTextString(m) } func (*ZLabelSet) ProtoMessage() {} func (*ZLabelSet) Descriptor() ([]byte, []int) { - return fileDescriptor_cdcc9e7dae4870e8, []int{2} + return fileDescriptor_cdcc9e7dae4870e8, []int{3} } func (m *ZLabelSet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -138,6 +176,7 @@ var xxx_messageInfo_ZLabelSet proto.InternalMessageInfo func init() { proto.RegisterType((*Label)(nil), "thanos.Label") + proto.RegisterType((*CompressedLabel)(nil), "thanos.CompressedLabel") proto.RegisterType((*LabelSet)(nil), "thanos.LabelSet") proto.RegisterType((*ZLabelSet)(nil), "thanos.ZLabelSet") } @@ -145,21 +184,24 @@ func init() { func init() { proto.RegisterFile("store/labelpb/types.proto", fileDescriptor_cdcc9e7dae4870e8) } var fileDescriptor_cdcc9e7dae4870e8 = []byte{ - // 212 bytes of a gzipped FileDescriptorProto + // 264 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2c, 0x2e, 0xc9, 0x2f, 0x4a, 0xd5, 0xcf, 0x49, 0x4c, 0x4a, 0xcd, 0x29, 0x48, 0xd2, 0x2f, 0xa9, 0x2c, 0x48, 0x2d, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x2b, 0xc9, 0x48, 0xcc, 0xcb, 0x2f, 0x96, 0x12, 0x49, 0xcf, 0x4f, 0xcf, 0x07, 0x0b, 0xe9, 0x83, 0x58, 0x10, 0x59, 0x25, 0x43, 0x2e, 0x56, 0x1f, 0x90, 0x26, 0x21, 0x21, 0x2e, 0x96, 0xbc, 0xc4, 0xdc, 0x54, 0x09, 0x46, 0x05, 0x46, 0x0d, 0xce, 0x20, 0x30, 0x5b, 0x48, 0x84, 0x8b, 0xb5, 0x2c, 0x31, 0xa7, 0x34, 0x55, 0x82, 0x09, 0x2c, 0x08, 0xe1, - 0x28, 0x99, 0x73, 0x71, 0x80, 0xb5, 0x04, 0xa7, 0x96, 0x08, 0x69, 0x73, 0xb1, 0x81, 0xed, 0x2c, - 0x96, 0x60, 0x54, 0x60, 0xd6, 0xe0, 0x36, 0xe2, 0xd5, 0x83, 0xd8, 0xa6, 0x07, 0x56, 0xe1, 0xc4, - 0x72, 0xe2, 0x9e, 0x3c, 0x43, 0x10, 0x54, 0x89, 0x92, 0x13, 0x17, 0x67, 0x14, 0x5c, 0xa7, 0x29, - 0x7e, 0x9d, 0x7c, 0x20, 0x9d, 0xb7, 0xee, 0xc9, 0xb3, 0x41, 0x74, 0xc0, 0xcc, 0x70, 0x52, 0x3d, - 0xf1, 0x50, 0x8e, 0xe1, 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, 0x92, 0x63, - 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, 0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, 0xd8, 0xa1, - 0x01, 0x90, 0xc4, 0x06, 0xf6, 0x9d, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x80, 0xe8, 0x16, - 0x18, 0x01, 0x00, 0x00, + 0x28, 0x79, 0x72, 0xf1, 0x3b, 0xe7, 0xe7, 0x16, 0x14, 0xa5, 0x16, 0x17, 0xa7, 0xa6, 0x40, 0x34, + 0x4b, 0x72, 0x71, 0x80, 0x34, 0xc4, 0x17, 0xa5, 0xa6, 0x81, 0x0d, 0x60, 0x09, 0x62, 0x07, 0xf1, + 0x83, 0x52, 0xd3, 0x84, 0xa4, 0xb9, 0x38, 0xc1, 0xda, 0xc0, 0x72, 0x4c, 0x60, 0x39, 0x0e, 0xb0, + 0x40, 0x50, 0x6a, 0x9a, 0x92, 0x39, 0x17, 0x07, 0xd8, 0x80, 0xe0, 0xd4, 0x12, 0x21, 0x6d, 0x2e, + 0x36, 0xb0, 0xf3, 0x8b, 0x25, 0x18, 0x15, 0x98, 0x35, 0xb8, 0x8d, 0x78, 0xf5, 0x20, 0x0e, 0xd7, + 0x03, 0xab, 0x70, 0x62, 0x39, 0x71, 0x4f, 0x9e, 0x21, 0x08, 0xaa, 0x44, 0xc9, 0x89, 0x8b, 0x33, + 0x0a, 0xae, 0xd3, 0x14, 0xbf, 0x4e, 0x3e, 0x90, 0xce, 0x5b, 0xf7, 0xe4, 0xd9, 0x20, 0x3a, 0x60, + 0x66, 0x38, 0xa9, 0x9e, 0x78, 0x28, 0xc7, 0x70, 0xe2, 0x91, 0x1c, 0xe3, 0x85, 0x47, 0x72, 0x8c, + 0x0f, 0x1e, 0xc9, 0x31, 0x4e, 0x78, 0x2c, 0xc7, 0x70, 0xe1, 0xb1, 0x1c, 0xc3, 0x8d, 0xc7, 0x72, + 0x0c, 0x51, 0xec, 0xd0, 0xb0, 0x4c, 0x62, 0x03, 0x07, 0x94, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, + 0x40, 0x5a, 0x96, 0x56, 0x63, 0x01, 0x00, 0x00, } func (m *Label) Marshal() (dAtA []byte, err error) { @@ -199,6 +241,39 @@ func (m *Label) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *CompressedLabel) 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 *CompressedLabel) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CompressedLabel) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.ValueRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.ValueRef)) + i-- + dAtA[i] = 0x10 + } + if m.NameRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.NameRef)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *LabelSet) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -301,6 +376,21 @@ func (m *Label) Size() (n int) { return n } +func (m *CompressedLabel) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NameRef != 0 { + n += 1 + sovTypes(uint64(m.NameRef)) + } + if m.ValueRef != 0 { + n += 1 + sovTypes(uint64(m.ValueRef)) + } + return n +} + func (m *LabelSet) Size() (n int) { if m == nil { return 0 @@ -451,6 +541,94 @@ func (m *Label) Unmarshal(dAtA []byte) error { } return nil } +func (m *CompressedLabel) 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: CompressedLabel: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CompressedLabel: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NameRef", wireType) + } + m.NameRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NameRef |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValueRef", wireType) + } + m.ValueRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ValueRef |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *LabelSet) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/store/labelpb/types.proto b/pkg/store/labelpb/types.proto index 65aa195ec1..bf9efce07b 100644 --- a/pkg/store/labelpb/types.proto +++ b/pkg/store/labelpb/types.proto @@ -24,6 +24,11 @@ message Label { string value = 2; } +message CompressedLabel { + uint64 name_ref = 1; + uint64 value_ref = 2; +} + message LabelSet { repeated Label labels = 1 [(gogoproto.nullable) = false]; } diff --git a/pkg/store/prometheus.go b/pkg/store/prometheus.go index b4546c7d05..6b733a8afa 100644 --- a/pkg/store/prometheus.go +++ b/pkg/store/prometheus.go @@ -23,6 +23,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/protobuf/proto" + "github.com/gogo/protobuf/types" "github.com/golang/snappy" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -35,6 +36,7 @@ import ( "github.com/thanos-io/thanos/pkg/httpconfig" "github.com/thanos-io/thanos/pkg/promclient" "github.com/thanos-io/thanos/pkg/runutil" + "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/thanos-io/thanos/pkg/store/storepb/prompb" @@ -174,6 +176,8 @@ func (p *PrometheusStore) Series(r *storepb.SeriesRequest, s storepb.Store_Serie } } + symbolTableBuilder := newSymbolTableBuilder(r.MaximumStringSlots) + if r.SkipChunks { labelMaps, err := p.client.SeriesInGRPC(s.Context(), p.base, matchers, r.MinTime, r.MaxTime) if err != nil { @@ -188,10 +192,27 @@ func (p *PrometheusStore) Series(r *storepb.SeriesRequest, s storepb.Store_Serie sort.Slice(lset, func(i, j int) bool { return lset[i].Name < lset[j].Name }) + if err = s.Send(storepb.NewSeriesResponse(&storepb.Series{Labels: lset})); err != nil { return err } } + + { + var anyHints *types.Any + + resHints := &hintspb.SeriesResponseHints{StringSymbolTable: symbolTableBuilder.getTable()} + + if anyHints, err = types.MarshalAny(resHints); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "marshal series response hints").Error()) + return err + } + + if err = s.Send(storepb.NewHintsSeriesResponse(anyHints)); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "send series response hints").Error()) + return err + } + } return nil } @@ -240,7 +261,7 @@ func (p *PrometheusStore) Series(r *storepb.SeriesRequest, s storepb.Store_Serie if !strings.HasPrefix(contentType, "application/x-streamed-protobuf; proto=prometheus.ChunkedReadResponse") { return errors.Errorf("not supported remote read content type: %s", contentType) } - return p.handleStreamedPrometheusResponse(s, shardMatcher, httpResp, queryPrometheusSpan, extLset, enableChunkHashCalculation) + return p.handleStreamedPrometheusResponse(s, shardMatcher, httpResp, queryPrometheusSpan, extLset, enableChunkHashCalculation, symbolTableBuilder) } func (p *PrometheusStore) queryPrometheus(s storepb.Store_SeriesServer, r *storepb.SeriesRequest) error { @@ -367,6 +388,7 @@ func (p *PrometheusStore) handleStreamedPrometheusResponse( querySpan tracing.Span, extLset labels.Labels, calculateChecksums bool, + lookupTable *symbolTableBuilder, ) error { level.Debug(p.logger).Log("msg", "started handling ReadRequest_STREAMED_XOR_CHUNKS streamed read response.") @@ -410,7 +432,24 @@ func (p *PrometheusStore) handleStreamedPrometheusResponse( continue } - seriesStats.CountSeries(series.Labels) + compressedLabels := make([]labelpb.CompressedLabel, 0, len(completeLabelset)) + var compressedResponse bool = true + for _, lbl := range completeLabelset { + nameRef, nok := lookupTable.getOrStoreString(lbl.Name) + valueRef, vok := lookupTable.getOrStoreString(lbl.Value) + + if !nok || !vok { + compressedResponse = false + break + } else if compressedResponse && nok && vok { + compressedLabels = append(compressedLabels, labelpb.CompressedLabel{ + NameRef: nameRef, + ValueRef: valueRef, + }) + } + } + + seriesStats.CountSeries(labelpb.HashWithPrefix("", series.Labels)) thanosChks := make([]storepb.AggrChunk, len(series.Chunks)) for i, chk := range series.Chunks { @@ -434,18 +473,43 @@ func (p *PrometheusStore) handleStreamedPrometheusResponse( series.Chunks[i].Data = nil } - r := storepb.NewSeriesResponse(&storepb.Series{ - Labels: labelpb.ZLabelsFromPromLabels( - completeLabelset, - ), - Chunks: thanosChks, - }) + var r *storepb.SeriesResponse + + if compressedResponse { + r = storepb.NewCompressedSeriesResponse(&storepb.CompressedSeries{ + Chunks: thanosChks, + Labels: compressedLabels, + }) + } else { + r = storepb.NewSeriesResponse(&storepb.Series{ + Labels: labelpb.ZLabelsFromPromLabels(completeLabelset), + Chunks: thanosChks, + }) + } + if err := s.Send(r); err != nil { return err } } } + { + var anyHints *types.Any + var err error + + resHints := &hintspb.SeriesResponseHints{StringSymbolTable: lookupTable.getTable()} + + if anyHints, err = types.MarshalAny(resHints); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "marshal series response hints").Error()) + return err + } + + if err = s.Send(storepb.NewHintsSeriesResponse(anyHints)); err != nil { + err = status.Error(codes.Unknown, errors.Wrap(err, "send series response hints").Error()) + return err + } + } + querySpan.SetTag("processed.series", seriesStats.Series) querySpan.SetTag("processed.chunks", seriesStats.Chunks) querySpan.SetTag("processed.samples", seriesStats.Samples) diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index d969ddb64f..71e510085e 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -71,9 +71,10 @@ type ProxyStore struct { selectorLabels labels.Labels buffers sync.Pool - responseTimeout time.Duration - metrics *proxyStoreMetrics - retrievalStrategy RetrievalStrategy + responseTimeout time.Duration + metrics *proxyStoreMetrics + retrievalStrategy RetrievalStrategy + enableCompressedRetrieval bool } type proxyStoreMetrics struct { @@ -107,6 +108,7 @@ func NewProxyStore( selectorLabels labels.Labels, responseTimeout time.Duration, retrievalStrategy RetrievalStrategy, + enableCompressedRetrieval bool, ) *ProxyStore { if logger == nil { logger = log.NewNopLogger() @@ -122,9 +124,10 @@ func NewProxyStore( b := make([]byte, 0, initialBufSize) return &b }}, - responseTimeout: responseTimeout, - metrics: metrics, - retrievalStrategy: retrievalStrategy, + responseTimeout: responseTimeout, + metrics: metrics, + retrievalStrategy: retrievalStrategy, + enableCompressedRetrieval: enableCompressedRetrieval, } return s } @@ -287,14 +290,21 @@ func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb. return nil } + // Zero maximum slots indicated that we want uncompressed data. + if s.enableCompressedRetrieval { + r.MaximumStringSlots = maxStringsPerStore(uint64(len(stores))) + } + adjusterFactory := newReferenceAdjusterFactory(uint64(len(stores))) + storeResponses := make([]respSet, 0, len(stores)) - for _, st := range stores { + for storeIndex, st := range stores { st := st storeDebugMsgs = append(storeDebugMsgs, fmt.Sprintf("store %s queried", st)) - respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, st.SupportsSharding(), &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses) + adjuster := adjusterFactory(uint64(storeIndex)) + respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, st.SupportsSharding(), &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses, adjuster) if err != nil { level.Error(reqLogger).Log("err", err) diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index 5cdb5a0b78..8f5fa85658 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -13,6 +13,7 @@ import ( "time" "github.com/go-kit/log" + "github.com/gogo/protobuf/types" "github.com/cespare/xxhash/v2" "github.com/go-kit/log/level" @@ -22,6 +23,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/labels" + "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/thanos-io/thanos/pkg/tracing" @@ -366,6 +368,8 @@ func newLazyRespSet( shardMatcher *storepb.ShardMatcher, applySharding bool, emptyStreamResponses prometheus.Counter, + maximumStrings uint64, + adjuster adjusterFn, ) respSet { bufferedResponses := []*storepb.SeriesResponse{} bufferedResponsesMtx := &sync.Mutex{} @@ -397,6 +401,7 @@ func newLazyRespSet( l.span.Finish() }() + var stringCnt uint64 numResponses := 0 defer func() { if numResponses == 0 { @@ -404,6 +409,15 @@ func newLazyRespSet( } }() + handleErr := func(e error) { + l.bufferedResponsesMtx.Lock() + l.bufferedResponses = append(l.bufferedResponses, storepb.NewWarnSeriesResponse(e)) + l.noMoreData = true + l.dataOrFinishEvent.Signal() + l.bufferedResponsesMtx.Unlock() + l.span.SetTag("err", e.Error()) + } + handleRecvResponse := func(t *time.Timer) bool { if t != nil { defer t.Reset(frameTimeout) @@ -411,14 +425,7 @@ func newLazyRespSet( 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() + handleErr(errors.Wrapf(l.ctx.Err(), "failed to receive any data from %s", st)) return false default: resp, err := cl.Recv() @@ -441,13 +448,7 @@ func newLazyRespSet( 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() + handleErr(rerr) return false } @@ -462,6 +463,50 @@ func newLazyRespSet( seriesStats.Count(resp.GetSeries()) } + if resp.GetCompressedSeries() != nil { + cs := resp.GetCompressedSeries() + seriesStats.Count(cs) + + stringCnt += uint64(len(cs.Labels)) + + if stringCnt > maximumStrings { + handleErr(fmt.Errorf("maximum string limit %d exceeded", maximumStrings)) + return false + } + + for i := range cs.Labels { + cs.Labels[i].NameRef = adjuster(cs.Labels[i].NameRef) + cs.Labels[i].ValueRef = adjuster(cs.Labels[i].ValueRef) + } + } + + if resp.GetHints() != nil { + var seriesResponseHints hintspb.SeriesResponseHints + var anyHints *types.Any + + if err := types.UnmarshalAny(resp.GetHints(), &seriesResponseHints); err != nil { + handleErr(err) + return false + } + + adjustedTable := map[uint64]string{} + + for k, v := range seriesResponseHints.StringSymbolTable { + adjustedTable[adjuster(k)] = v + } + + seriesResponseHints.StringSymbolTable = adjustedTable + + resHints := &hintspb.SeriesResponseHints{StringSymbolTable: adjustedTable, QueriedBlocks: seriesResponseHints.QueriedBlocks} + + if anyHints, err = types.MarshalAny(resHints); err != nil { + handleErr(err) + return false + } + + resp = storepb.NewHintsSeriesResponse(anyHints) + } + l.bufferedResponsesMtx.Lock() l.bufferedResponses = append(l.bufferedResponses, resp) l.dataOrFinishEvent.Signal() @@ -507,7 +552,9 @@ func newAsyncRespSet(ctx context.Context, buffers *sync.Pool, shardInfo *storepb.ShardInfo, logger log.Logger, - emptyStreamResponses prometheus.Counter) (respSet, error) { + emptyStreamResponses prometheus.Counter, + adjuster adjusterFn, +) (respSet, error) { var span opentracing.Span var closeSeries context.CancelFunc @@ -561,6 +608,8 @@ func newAsyncRespSet(ctx context.Context, shardMatcher, applySharding, emptyStreamResponses, + req.MaximumStringSlots, + adjuster, ), nil case EagerRetrieval: return newEagerRespSet( @@ -573,6 +622,8 @@ func newAsyncRespSet(ctx context.Context, shardMatcher, applySharding, emptyStreamResponses, + req.MaximumStringSlots, + adjuster, ), nil default: panic(fmt.Sprintf("unsupported retrieval strategy %s", retrievalStrategy)) @@ -620,6 +671,8 @@ func newEagerRespSet( shardMatcher *storepb.ShardMatcher, applySharding bool, emptyStreamResponses prometheus.Counter, + maximumStrings uint64, + adjuster adjusterFn, ) respSet { ret := &eagerRespSet{ span: span, @@ -650,6 +703,7 @@ func newEagerRespSet( }() numResponses := 0 + var stringCnt uint64 defer func() { if numResponses == 0 { emptyStreamResponses.Inc() @@ -698,6 +752,51 @@ func newEagerRespSet( seriesStats.Count(resp.GetSeries()) } + if resp.GetCompressedSeries() != nil { + cs := resp.GetCompressedSeries() + seriesStats.Count(cs) + + stringCnt += uint64(len(cs.Labels)) + + if stringCnt > maximumStrings { + err := fmt.Errorf("maximum string limit %d exceeded", maximumStrings) + l.span.SetTag("err", err.Error()) + return false + } + + for i := range cs.Labels { + cs.Labels[i].NameRef = adjuster(cs.Labels[i].NameRef) + cs.Labels[i].ValueRef = adjuster(cs.Labels[i].ValueRef) + } + } + + if resp.GetHints() != nil { + var seriesResponseHints hintspb.SeriesResponseHints + var anyHints *types.Any + + if err := types.UnmarshalAny(resp.GetHints(), &seriesResponseHints); err != nil { + l.span.SetTag("err", err.Error()) + return false + } + + adjustedTable := map[uint64]string{} + + for k, v := range seriesResponseHints.StringSymbolTable { + adjustedTable[adjuster(k)] = v + } + + seriesResponseHints.StringSymbolTable = adjustedTable + + resHints := &hintspb.SeriesResponseHints{StringSymbolTable: adjustedTable, QueriedBlocks: seriesResponseHints.QueriedBlocks} + + if anyHints, err = types.MarshalAny(resHints); err != nil { + l.span.SetTag("err", err.Error()) + return false + } + + resp = storepb.NewHintsSeriesResponse(anyHints) + } + l.bufferedResponses = append(l.bufferedResponses, resp) return true } diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index 7c1064233f..5adced37db 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -101,6 +101,7 @@ func TestProxyStore_Info(t *testing.T) { func() []Client { return nil }, component.Query, nil, 0*time.Second, RetrievalStrategy(EagerRetrieval), + false, ) resp, err := q.Info(ctx, &storepb.InfoRequest{}) @@ -548,6 +549,7 @@ func TestProxyStore_Series(t *testing.T) { component.Query, tc.selectorLabels, 5*time.Second, strategy, + false, ) ctx := context.Background() @@ -1082,6 +1084,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { component.Query, tc.selectorLabels, 4*time.Second, strategy, + false, ) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) @@ -1140,6 +1143,7 @@ func TestProxyStore_Series_RequestParamsProxied(t *testing.T) { component.Query, nil, 1*time.Second, EagerRetrieval, + false, ) ctx := context.Background() @@ -1156,6 +1160,7 @@ func TestProxyStore_Series_RequestParamsProxied(t *testing.T) { }, PartialResponseStrategy: storepb.PartialResponseStrategy_WARN, MaxResolutionWindow: 1234, + MaximumStringSlots: math.MaxUint64, } testutil.Ok(t, q.Series(req, s)) @@ -1201,6 +1206,7 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { component.Query, labels.FromStrings("fed", "a"), 5*time.Second, EagerRetrieval, + false, ) ctx := context.Background() @@ -1248,6 +1254,7 @@ func TestProxyStore_LabelValues(t *testing.T) { component.Query, nil, 0*time.Second, EagerRetrieval, + false, ) ctx := context.Background() @@ -1445,6 +1452,7 @@ func TestProxyStore_LabelNames(t *testing.T) { component.Query, nil, 5*time.Second, EagerRetrieval, + false, ) ctx := context.Background() diff --git a/pkg/store/storepb/custom.go b/pkg/store/storepb/custom.go index 85f858562b..89f045b524 100644 --- a/pkg/store/storepb/custom.go +++ b/pkg/store/storepb/custom.go @@ -46,6 +46,14 @@ func NewSeriesResponse(series *Series) *SeriesResponse { } } +func NewCompressedSeriesResponse(series *CompressedSeries) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_CompressedSeries{ + CompressedSeries: series, + }, + } +} + func NewHintsSeriesResponse(hints *types.Any) *SeriesResponse { return &SeriesResponse{ Result: &SeriesResponse_Hints{ @@ -488,17 +496,15 @@ type SeriesStatsCounter struct { Samples int } -func (c *SeriesStatsCounter) CountSeries(seriesLabels []labelpb.ZLabel) { - seriesHash := labelpb.HashWithPrefix("", seriesLabels) - if c.lastSeriesHash != 0 || seriesHash != c.lastSeriesHash { - c.lastSeriesHash = seriesHash +func (c *SeriesStatsCounter) CountSeries(seriesLabelsHash uint64) { + if c.lastSeriesHash != 0 || seriesLabelsHash != c.lastSeriesHash { + c.lastSeriesHash = seriesLabelsHash c.Series++ } } -func (c *SeriesStatsCounter) Count(series *Series) { - c.CountSeries(series.Labels) - for _, chk := range series.Chunks { +func (c *SeriesStatsCounter) countChks(chks []AggrChunk) { + for _, chk := range chks { if chk.Raw != nil { c.Chunks++ c.Samples += chk.Raw.XORNumSamples() @@ -531,6 +537,25 @@ func (c *SeriesStatsCounter) Count(series *Series) { } } +func (s *CompressedSeries) Count(counter *SeriesStatsCounter) { + counter.CountSeries(labelpb.HashCompressedWithPrefix("", s.Labels)) + counter.countChks(s.Chunks) + +} + +func (s *Series) Count(counter *SeriesStatsCounter) { + counter.CountSeries(labelpb.HashWithPrefix("", s.Labels)) + counter.countChks(s.Chunks) +} + +type countableSeries interface { + Count(*SeriesStatsCounter) +} + +func (c *SeriesStatsCounter) Count(series countableSeries) { + series.Count(c) +} + func (m *SeriesRequest) ToPromQL() string { return m.QueryHints.toPromQL(m.Matchers) } diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index bf670d0f63..9d647efc9a 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -292,6 +292,11 @@ type SeriesRequest struct { // 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"` + // maximum_string_slots is the number of maximum strings + // that the receiver can send. It is safe to ignore this by + // the receiver if it does not intend on sending a symbol table + // via hints later on. + MaximumStringSlots uint64 `protobuf:"varint,14,opt,name=maximum_string_slots,json=maximumStringSlots,proto3" json:"maximum_string_slots,omitempty"` } func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } @@ -537,6 +542,7 @@ type SeriesResponse struct { // *SeriesResponse_Series // *SeriesResponse_Warning // *SeriesResponse_Hints + // *SeriesResponse_CompressedSeries Result isSeriesResponse_Result `protobuf_oneof:"result"` } @@ -588,10 +594,14 @@ type SeriesResponse_Warning struct { type SeriesResponse_Hints struct { Hints *types.Any `protobuf:"bytes,3,opt,name=hints,proto3,oneof" json:"hints,omitempty"` } +type SeriesResponse_CompressedSeries struct { + CompressedSeries *CompressedSeries `protobuf:"bytes,4,opt,name=compressed_series,json=compressedSeries,proto3,oneof" json:"compressed_series,omitempty"` +} -func (*SeriesResponse_Series) isSeriesResponse_Result() {} -func (*SeriesResponse_Warning) isSeriesResponse_Result() {} -func (*SeriesResponse_Hints) isSeriesResponse_Result() {} +func (*SeriesResponse_Series) isSeriesResponse_Result() {} +func (*SeriesResponse_Warning) isSeriesResponse_Result() {} +func (*SeriesResponse_Hints) isSeriesResponse_Result() {} +func (*SeriesResponse_CompressedSeries) isSeriesResponse_Result() {} func (m *SeriesResponse) GetResult() isSeriesResponse_Result { if m != nil { @@ -621,12 +631,20 @@ func (m *SeriesResponse) GetHints() *types.Any { return nil } +func (m *SeriesResponse) GetCompressedSeries() *CompressedSeries { + if x, ok := m.GetResult().(*SeriesResponse_CompressedSeries); ok { + return x.CompressedSeries + } + 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), + (*SeriesResponse_CompressedSeries)(nil), } } @@ -830,89 +848,92 @@ func init() { func init() { proto.RegisterFile("store/storepb/rpc.proto", fileDescriptor_a938d55a388af629) } var fileDescriptor_a938d55a388af629 = []byte{ - // 1298 bytes of a gzipped FileDescriptorProto + // 1358 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x5d, 0x6f, 0x13, 0x47, - 0x17, 0xf6, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x9a, 0xc1, 0xc0, 0xc6, 0x48, 0x8e, 0xdf, 0x7d, - 0xf5, 0x4a, 0x11, 0xa2, 0x36, 0x35, 0x15, 0x52, 0x2b, 0x6e, 0x92, 0x60, 0x48, 0x54, 0x62, 0xca, - 0x38, 0x21, 0x2d, 0x55, 0x65, 0xad, 0xed, 0xc9, 0x7a, 0xc5, 0x7a, 0x77, 0xd9, 0x99, 0x6d, 0xe2, - 0xdb, 0xf6, 0xbe, 0xaa, 0xfa, 0x13, 0xfa, 0x2b, 0xfa, 0x13, 0xb8, 0x2b, 0x57, 0x55, 0xd5, 0x0b, - 0xd4, 0xc2, 0x1f, 0xa9, 0xe6, 0x63, 0xd7, 0xde, 0x34, 0x40, 0x11, 0xdc, 0x44, 0x73, 0x9e, 0xe7, - 0xcc, 0x99, 0xf3, 0xed, 0x0d, 0x5c, 0xa1, 0x2c, 0x88, 0x48, 0x47, 0xfc, 0x0d, 0x47, 0x9d, 0x28, - 0x1c, 0xb7, 0xc3, 0x28, 0x60, 0x01, 0x2a, 0xb2, 0xa9, 0xed, 0x07, 0xb4, 0xb1, 0x9e, 0x55, 0x60, - 0xf3, 0x90, 0x50, 0xa9, 0xd2, 0xa8, 0x3b, 0x81, 0x13, 0x88, 0x63, 0x87, 0x9f, 0x14, 0xda, 0xca, - 0x5e, 0x08, 0xa3, 0x60, 0x76, 0xe6, 0x9e, 0x32, 0xe9, 0xd9, 0x23, 0xe2, 0x9d, 0xa5, 0x9c, 0x20, - 0x70, 0x3c, 0xd2, 0x11, 0xd2, 0x28, 0x3e, 0xee, 0xd8, 0xfe, 0x5c, 0x52, 0xd6, 0x7f, 0x60, 0xf5, - 0x28, 0x72, 0x19, 0xc1, 0x84, 0x86, 0x81, 0x4f, 0x89, 0xf5, 0xbd, 0x06, 0x2b, 0x0a, 0x79, 0x1a, - 0x13, 0xca, 0xd0, 0x16, 0x00, 0x73, 0x67, 0x84, 0x92, 0xc8, 0x25, 0xd4, 0xd4, 0x5a, 0xfa, 0x66, - 0xb5, 0x7b, 0x95, 0xdf, 0x9e, 0x11, 0x36, 0x25, 0x31, 0x1d, 0x8e, 0x83, 0x70, 0xde, 0x3e, 0x70, - 0x67, 0x64, 0x20, 0x54, 0xb6, 0x0b, 0xcf, 0x5e, 0x6c, 0xe4, 0xf0, 0xd2, 0x25, 0x74, 0x19, 0x8a, - 0x8c, 0xf8, 0xb6, 0xcf, 0xcc, 0x7c, 0x4b, 0xdb, 0xac, 0x60, 0x25, 0x21, 0x13, 0x4a, 0x11, 0x09, - 0x3d, 0x77, 0x6c, 0x9b, 0x7a, 0x4b, 0xdb, 0xd4, 0x71, 0x22, 0x5a, 0xab, 0x50, 0xdd, 0xf3, 0x8f, - 0x03, 0xe5, 0x83, 0xf5, 0x53, 0x1e, 0x56, 0xa4, 0x2c, 0xbd, 0x44, 0x63, 0x28, 0x8a, 0x40, 0x13, - 0x87, 0x56, 0xdb, 0x32, 0xb1, 0xed, 0xfb, 0x1c, 0xdd, 0xbe, 0xcd, 0x5d, 0xf8, 0xe3, 0xc5, 0xc6, - 0x27, 0x8e, 0xcb, 0xa6, 0xf1, 0xa8, 0x3d, 0x0e, 0x66, 0x1d, 0xa9, 0xf0, 0x91, 0x1b, 0xa8, 0x53, - 0x27, 0x7c, 0xe2, 0x74, 0x32, 0x39, 0x6b, 0x3f, 0x16, 0xb7, 0xb1, 0x32, 0x8d, 0xd6, 0xa1, 0x3c, - 0x73, 0xfd, 0x21, 0x0f, 0x44, 0x38, 0xae, 0xe3, 0xd2, 0xcc, 0xf5, 0x79, 0xa4, 0x82, 0xb2, 0x4f, - 0x25, 0xa5, 0x5c, 0x9f, 0xd9, 0xa7, 0x82, 0xea, 0x40, 0x45, 0x58, 0x3d, 0x98, 0x87, 0xc4, 0x2c, - 0xb4, 0xb4, 0xcd, 0xb5, 0xee, 0x85, 0xc4, 0xbb, 0x41, 0x42, 0xe0, 0x85, 0x0e, 0xba, 0x05, 0x20, - 0x1e, 0x1c, 0x52, 0xc2, 0xa8, 0x69, 0x88, 0x78, 0xd2, 0x1b, 0xd2, 0xa5, 0x01, 0x61, 0x2a, 0xad, - 0x15, 0x4f, 0xc9, 0xd4, 0xfa, 0xad, 0x00, 0xab, 0x32, 0xe5, 0x49, 0xa9, 0x96, 0x1d, 0xd6, 0x5e, - 0xef, 0x70, 0x3e, 0xeb, 0xf0, 0x2d, 0x4e, 0xb1, 0xf1, 0x94, 0x44, 0xd4, 0xd4, 0xc5, 0xeb, 0xf5, - 0x4c, 0x36, 0xf7, 0x25, 0xa9, 0x1c, 0x48, 0x75, 0x51, 0x17, 0x2e, 0x71, 0x93, 0x11, 0xa1, 0x81, - 0x17, 0x33, 0x37, 0xf0, 0x87, 0x27, 0xae, 0x3f, 0x09, 0x4e, 0x44, 0xd0, 0x3a, 0xbe, 0x38, 0xb3, - 0x4f, 0x71, 0xca, 0x1d, 0x09, 0x0a, 0x5d, 0x07, 0xb0, 0x1d, 0x27, 0x22, 0x8e, 0xcd, 0x88, 0x8c, - 0x75, 0xad, 0xbb, 0x92, 0xbc, 0xb6, 0xe5, 0x38, 0x11, 0x5e, 0xe2, 0xd1, 0x67, 0xb0, 0x1e, 0xda, - 0x11, 0x73, 0x6d, 0x8f, 0xbf, 0x22, 0x2a, 0x3f, 0x9c, 0xb8, 0xd4, 0x1e, 0x79, 0x64, 0x62, 0x16, - 0x5b, 0xda, 0x66, 0x19, 0x5f, 0x51, 0x0a, 0x49, 0x67, 0xdc, 0x51, 0x34, 0xfa, 0xfa, 0x9c, 0xbb, - 0x94, 0x45, 0x36, 0x23, 0xce, 0xdc, 0x2c, 0x89, 0xb2, 0x6c, 0x24, 0x0f, 0x7f, 0x91, 0xb5, 0x31, - 0x50, 0x6a, 0xff, 0x30, 0x9e, 0x10, 0x68, 0x03, 0xaa, 0xf4, 0x89, 0x1b, 0x0e, 0xc7, 0xd3, 0xd8, - 0x7f, 0x42, 0xcd, 0xb2, 0x70, 0x05, 0x38, 0xb4, 0x23, 0x10, 0x74, 0x0d, 0x8c, 0xa9, 0xeb, 0x33, - 0x6a, 0x56, 0x5a, 0x9a, 0x48, 0xa8, 0x9c, 0xc0, 0x76, 0x32, 0x81, 0xed, 0x2d, 0x7f, 0x8e, 0xa5, - 0x0a, 0x42, 0x50, 0xa0, 0x8c, 0x84, 0x26, 0x88, 0xb4, 0x89, 0x33, 0xaa, 0x83, 0x11, 0xd9, 0xbe, - 0x43, 0xcc, 0xaa, 0x00, 0xa5, 0x80, 0x6e, 0x42, 0xf5, 0x69, 0x4c, 0xa2, 0xf9, 0x50, 0xda, 0x5e, - 0x11, 0xb6, 0x51, 0x12, 0xc5, 0x43, 0x4e, 0xed, 0x72, 0x06, 0xc3, 0xd3, 0xf4, 0x8c, 0x6e, 0x00, - 0xd0, 0xa9, 0x1d, 0x4d, 0x86, 0xae, 0x7f, 0x1c, 0x98, 0xab, 0xe2, 0xce, 0xa2, 0x21, 0x39, 0x23, - 0x26, 0xab, 0x42, 0x93, 0xa3, 0xf5, 0xb3, 0x06, 0xb0, 0x30, 0x26, 0x82, 0x65, 0x24, 0x1c, 0xce, - 0x5c, 0xcf, 0x73, 0xa9, 0x6a, 0x2c, 0xe0, 0xd0, 0xbe, 0x40, 0x50, 0x0b, 0x0a, 0xc7, 0xb1, 0x3f, - 0x16, 0x7d, 0x55, 0x5d, 0x94, 0xf3, 0x6e, 0xec, 0x8f, 0xb1, 0x60, 0xd0, 0x75, 0x28, 0x3b, 0x51, - 0x10, 0x87, 0xae, 0xef, 0x88, 0xee, 0xa8, 0x76, 0x6b, 0x89, 0xd6, 0x3d, 0x85, 0xe3, 0x54, 0x03, - 0xfd, 0x2f, 0x09, 0xde, 0x10, 0xaa, 0xe9, 0x6c, 0x63, 0x0e, 0xaa, 0x5c, 0x58, 0x27, 0x50, 0x49, - 0x9d, 0x17, 0x2e, 0xaa, 0x18, 0x27, 0xe4, 0x34, 0x75, 0x51, 0xf2, 0x13, 0x72, 0x8a, 0xfe, 0x0b, - 0x2b, 0x2c, 0x60, 0xb6, 0x37, 0x14, 0x18, 0x55, 0x23, 0x50, 0x15, 0x98, 0x30, 0x43, 0xd1, 0x1a, - 0xe4, 0x47, 0x73, 0x31, 0xcc, 0x65, 0x9c, 0x1f, 0xcd, 0xf9, 0xd2, 0x52, 0x2b, 0xa6, 0xd0, 0xd2, - 0xf9, 0xd2, 0x92, 0x92, 0xd5, 0x80, 0x02, 0x8f, 0x8c, 0x97, 0xcd, 0xb7, 0xd5, 0xa0, 0x55, 0xb0, - 0x38, 0x5b, 0x5d, 0x28, 0x27, 0xf1, 0x28, 0x7b, 0xda, 0x39, 0xf6, 0xf4, 0x8c, 0xbd, 0x0d, 0x30, - 0x44, 0x60, 0x5c, 0x21, 0x93, 0x62, 0x25, 0x59, 0x3f, 0x68, 0xb0, 0x96, 0xcc, 0xb9, 0x5a, 0x7f, - 0x9b, 0x50, 0x4c, 0xf7, 0x31, 0x4f, 0xd1, 0x5a, 0x5a, 0x4f, 0x81, 0xee, 0xe6, 0xb0, 0xe2, 0x51, - 0x03, 0x4a, 0x27, 0x76, 0xe4, 0xf3, 0xc4, 0x8b, 0xdd, 0xbb, 0x9b, 0xc3, 0x09, 0x80, 0xae, 0x27, - 0x4d, 0xaa, 0xbf, 0xbe, 0x49, 0x77, 0x73, 0xaa, 0x4d, 0xb7, 0xcb, 0x50, 0x8c, 0x08, 0x8d, 0x3d, - 0x66, 0xfd, 0x92, 0x87, 0x0b, 0x62, 0x33, 0xf4, 0xed, 0xd9, 0x62, 0xf9, 0xbc, 0x71, 0x58, 0xb5, - 0xf7, 0x18, 0xd6, 0xfc, 0x7b, 0x0e, 0x6b, 0x1d, 0x0c, 0xca, 0xec, 0x88, 0xa9, 0x45, 0x2d, 0x05, - 0x54, 0x03, 0x9d, 0xf8, 0x13, 0xb5, 0xab, 0xf8, 0x71, 0x31, 0xb3, 0xc6, 0xdb, 0x67, 0x76, 0x79, - 0x67, 0x16, 0xff, 0xfd, 0xce, 0xb4, 0x22, 0x40, 0xcb, 0x99, 0x53, 0xe5, 0xac, 0x83, 0xc1, 0xdb, - 0x47, 0xfe, 0x98, 0x55, 0xb0, 0x14, 0x50, 0x03, 0xca, 0xaa, 0x52, 0xbc, 0x5f, 0x39, 0x91, 0xca, - 0x0b, 0x5f, 0xf5, 0xb7, 0xfa, 0x6a, 0xfd, 0x9a, 0x57, 0x8f, 0x3e, 0xb2, 0xbd, 0x78, 0x51, 0xaf, - 0x3a, 0x18, 0xa2, 0x03, 0x55, 0x03, 0x4b, 0xe1, 0xcd, 0x55, 0xcc, 0xbf, 0x47, 0x15, 0xf5, 0x0f, - 0x55, 0xc5, 0xc2, 0x39, 0x55, 0x34, 0xce, 0xa9, 0x62, 0xf1, 0xdd, 0xaa, 0x58, 0x7a, 0x87, 0x2a, - 0xc6, 0x70, 0x31, 0x93, 0x50, 0x55, 0xc6, 0xcb, 0x50, 0xfc, 0x56, 0x20, 0xaa, 0x8e, 0x4a, 0xfa, - 0x50, 0x85, 0xbc, 0xf6, 0x0d, 0x54, 0xd2, 0x0f, 0x08, 0x54, 0x85, 0xd2, 0x61, 0xff, 0xf3, 0xfe, - 0x83, 0xa3, 0x7e, 0x2d, 0x87, 0x2a, 0x60, 0x3c, 0x3c, 0xec, 0xe1, 0xaf, 0x6a, 0x1a, 0x2a, 0x43, - 0x01, 0x1f, 0xde, 0xef, 0xd5, 0xf2, 0x5c, 0x63, 0xb0, 0x77, 0xa7, 0xb7, 0xb3, 0x85, 0x6b, 0x3a, - 0xd7, 0x18, 0x1c, 0x3c, 0xc0, 0xbd, 0x5a, 0x81, 0xe3, 0xb8, 0xb7, 0xd3, 0xdb, 0x7b, 0xd4, 0xab, - 0x19, 0x1c, 0xbf, 0xd3, 0xdb, 0x3e, 0xbc, 0x57, 0x2b, 0x5e, 0xdb, 0x86, 0x02, 0xff, 0x05, 0x46, - 0x25, 0xd0, 0xf1, 0xd6, 0x91, 0xb4, 0xba, 0xf3, 0xe0, 0xb0, 0x7f, 0x50, 0xd3, 0x38, 0x36, 0x38, - 0xdc, 0xaf, 0xe5, 0xf9, 0x61, 0x7f, 0xaf, 0x5f, 0xd3, 0xc5, 0x61, 0xeb, 0x4b, 0x69, 0x4e, 0x68, - 0xf5, 0x70, 0xcd, 0xe8, 0x7e, 0x97, 0x07, 0x43, 0xf8, 0x88, 0x3e, 0x86, 0x82, 0x58, 0xcd, 0x17, - 0x93, 0x8c, 0x2e, 0x7d, 0xcf, 0x35, 0xea, 0x59, 0x50, 0xe5, 0xef, 0x53, 0x28, 0xca, 0xfd, 0x85, - 0x2e, 0x65, 0xf7, 0x59, 0x72, 0xed, 0xf2, 0x59, 0x58, 0x5e, 0xbc, 0xa1, 0xa1, 0x1d, 0x80, 0xc5, - 0x5c, 0xa1, 0xf5, 0x4c, 0x15, 0x97, 0xb7, 0x54, 0xa3, 0x71, 0x1e, 0xa5, 0xde, 0xbf, 0x0b, 0xd5, - 0xa5, 0xb2, 0xa2, 0xac, 0x6a, 0x66, 0x78, 0x1a, 0x57, 0xcf, 0xe5, 0xa4, 0x9d, 0x6e, 0x1f, 0xd6, - 0xc4, 0x17, 0x34, 0x9f, 0x0a, 0x99, 0x8c, 0xdb, 0x50, 0xc5, 0x64, 0x16, 0x30, 0x22, 0x70, 0x94, - 0x86, 0xbf, 0xfc, 0xa1, 0xdd, 0xb8, 0x74, 0x06, 0x55, 0x1f, 0xe4, 0xb9, 0xed, 0xff, 0x3f, 0xfb, - 0xab, 0x99, 0x7b, 0xf6, 0xb2, 0xa9, 0x3d, 0x7f, 0xd9, 0xd4, 0xfe, 0x7c, 0xd9, 0xd4, 0x7e, 0x7c, - 0xd5, 0xcc, 0x3d, 0x7f, 0xd5, 0xcc, 0xfd, 0xfe, 0xaa, 0x99, 0x7b, 0x5c, 0x52, 0xff, 0x13, 0x8c, - 0x8a, 0xa2, 0x67, 0x6e, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x02, 0x42, 0x0e, 0xd0, 0x7d, 0x0c, - 0x00, 0x00, + 0x17, 0xf6, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x2e, 0x83, 0x81, 0x8d, 0x91, 0x12, 0xbf, 0x7e, + 0xf5, 0x4a, 0x11, 0xa2, 0x36, 0x35, 0x15, 0x52, 0x2b, 0x6e, 0x92, 0x60, 0x48, 0x54, 0x12, 0xca, + 0x38, 0x21, 0x2d, 0x55, 0x65, 0xad, 0xed, 0xc9, 0x7a, 0xc5, 0x7e, 0xb1, 0x33, 0xdb, 0xc4, 0xb7, + 0xed, 0x4d, 0x2f, 0xab, 0xfe, 0x84, 0xfe, 0x8a, 0xfe, 0x04, 0xee, 0xca, 0x5d, 0xab, 0x5e, 0xa0, + 0x16, 0xfe, 0x48, 0x35, 0x1f, 0xbb, 0xf6, 0xa6, 0x01, 0x8a, 0xe0, 0x26, 0x9a, 0xf3, 0x3c, 0x67, + 0xce, 0x9c, 0x39, 0xe7, 0xcc, 0xe3, 0x0d, 0x5c, 0xa1, 0x2c, 0x8c, 0x49, 0x57, 0xfc, 0x8d, 0x46, + 0xdd, 0x38, 0x1a, 0x77, 0xa2, 0x38, 0x64, 0x21, 0x2a, 0xb3, 0xa9, 0x1d, 0x84, 0xb4, 0xb9, 0x9a, + 0x77, 0x60, 0xb3, 0x88, 0x50, 0xe9, 0xd2, 0x6c, 0x38, 0xa1, 0x13, 0x8a, 0x65, 0x97, 0xaf, 0x14, + 0xda, 0xca, 0x6f, 0x88, 0xe2, 0xd0, 0x3f, 0xb3, 0x4f, 0x85, 0xf4, 0xec, 0x11, 0xf1, 0xce, 0x52, + 0x4e, 0x18, 0x3a, 0x1e, 0xe9, 0x0a, 0x6b, 0x94, 0x1c, 0x77, 0xed, 0x60, 0x26, 0xa9, 0xf6, 0x7f, + 0x60, 0xf9, 0x28, 0x76, 0x19, 0xc1, 0x84, 0x46, 0x61, 0x40, 0x49, 0xfb, 0x7b, 0x0d, 0x96, 0x14, + 0xf2, 0x34, 0x21, 0x94, 0xa1, 0x4d, 0x00, 0xe6, 0xfa, 0x84, 0x92, 0xd8, 0x25, 0xd4, 0xd2, 0x5a, + 0xfa, 0x46, 0xbd, 0x77, 0x95, 0xef, 0xf6, 0x09, 0x9b, 0x92, 0x84, 0x0e, 0xc7, 0x61, 0x34, 0xeb, + 0x1c, 0xb8, 0x3e, 0x19, 0x08, 0x97, 0xad, 0xd2, 0xb3, 0x17, 0xeb, 0x05, 0xbc, 0xb0, 0x09, 0x5d, + 0x86, 0x32, 0x23, 0x81, 0x1d, 0x30, 0xab, 0xd8, 0xd2, 0x36, 0x6a, 0x58, 0x59, 0xc8, 0x82, 0x4a, + 0x4c, 0x22, 0xcf, 0x1d, 0xdb, 0x96, 0xde, 0xd2, 0x36, 0x74, 0x9c, 0x9a, 0xed, 0x65, 0xa8, 0xef, + 0x06, 0xc7, 0xa1, 0xca, 0xa1, 0xfd, 0x53, 0x11, 0x96, 0xa4, 0x2d, 0xb3, 0x44, 0x63, 0x28, 0x8b, + 0x8b, 0xa6, 0x09, 0x2d, 0x77, 0x64, 0x61, 0x3b, 0xf7, 0x39, 0xba, 0x75, 0x9b, 0xa7, 0xf0, 0xc7, + 0x8b, 0xf5, 0x4f, 0x1c, 0x97, 0x4d, 0x93, 0x51, 0x67, 0x1c, 0xfa, 0x5d, 0xe9, 0xf0, 0x91, 0x1b, + 0xaa, 0x55, 0x37, 0x7a, 0xe2, 0x74, 0x73, 0x35, 0xeb, 0x3c, 0x16, 0xbb, 0xb1, 0x0a, 0x8d, 0x56, + 0xa1, 0xea, 0xbb, 0xc1, 0x90, 0x5f, 0x44, 0x24, 0xae, 0xe3, 0x8a, 0xef, 0x06, 0xfc, 0xa6, 0x82, + 0xb2, 0x4f, 0x25, 0xa5, 0x52, 0xf7, 0xed, 0x53, 0x41, 0x75, 0xa1, 0x26, 0xa2, 0x1e, 0xcc, 0x22, + 0x62, 0x95, 0x5a, 0xda, 0xc6, 0x4a, 0xef, 0x42, 0x9a, 0xdd, 0x20, 0x25, 0xf0, 0xdc, 0x07, 0xdd, + 0x02, 0x10, 0x07, 0x0e, 0x29, 0x61, 0xd4, 0x32, 0xc4, 0x7d, 0xb2, 0x1d, 0x32, 0xa5, 0x01, 0x61, + 0xaa, 0xac, 0x35, 0x4f, 0xd9, 0xb4, 0xfd, 0x83, 0x01, 0xcb, 0xb2, 0xe4, 0x69, 0xab, 0x16, 0x13, + 0xd6, 0x5e, 0x9f, 0x70, 0x31, 0x9f, 0xf0, 0x2d, 0x4e, 0xb1, 0xf1, 0x94, 0xc4, 0xd4, 0xd2, 0xc5, + 0xe9, 0x8d, 0x5c, 0x35, 0xf7, 0x24, 0xa9, 0x12, 0xc8, 0x7c, 0x51, 0x0f, 0x2e, 0xf1, 0x90, 0x31, + 0xa1, 0xa1, 0x97, 0x30, 0x37, 0x0c, 0x86, 0x27, 0x6e, 0x30, 0x09, 0x4f, 0xc4, 0xa5, 0x75, 0x7c, + 0xd1, 0xb7, 0x4f, 0x71, 0xc6, 0x1d, 0x09, 0x0a, 0x5d, 0x07, 0xb0, 0x1d, 0x27, 0x26, 0x8e, 0xcd, + 0x88, 0xbc, 0xeb, 0x4a, 0x6f, 0x29, 0x3d, 0x6d, 0xd3, 0x71, 0x62, 0xbc, 0xc0, 0xa3, 0xcf, 0x60, + 0x35, 0xb2, 0x63, 0xe6, 0xda, 0x1e, 0x3f, 0x45, 0x74, 0x7e, 0x38, 0x71, 0xa9, 0x3d, 0xf2, 0xc8, + 0xc4, 0x2a, 0xb7, 0xb4, 0x8d, 0x2a, 0xbe, 0xa2, 0x1c, 0xd2, 0xc9, 0xb8, 0xa3, 0x68, 0xf4, 0xf5, + 0x39, 0x7b, 0x29, 0x8b, 0x6d, 0x46, 0x9c, 0x99, 0x55, 0x11, 0x6d, 0x59, 0x4f, 0x0f, 0xfe, 0x22, + 0x1f, 0x63, 0xa0, 0xdc, 0xfe, 0x11, 0x3c, 0x25, 0xd0, 0x3a, 0xd4, 0xe9, 0x13, 0x37, 0x1a, 0x8e, + 0xa7, 0x49, 0xf0, 0x84, 0x5a, 0x55, 0x91, 0x0a, 0x70, 0x68, 0x5b, 0x20, 0xe8, 0x1a, 0x18, 0x53, + 0x37, 0x60, 0xd4, 0xaa, 0xb5, 0x34, 0x51, 0x50, 0xf9, 0x02, 0x3b, 0xe9, 0x0b, 0xec, 0x6c, 0x06, + 0x33, 0x2c, 0x5d, 0x10, 0x82, 0x12, 0x65, 0x24, 0xb2, 0x40, 0x94, 0x4d, 0xac, 0x51, 0x03, 0x8c, + 0xd8, 0x0e, 0x1c, 0x62, 0xd5, 0x05, 0x28, 0x0d, 0x74, 0x13, 0xea, 0x4f, 0x13, 0x12, 0xcf, 0x86, + 0x32, 0xf6, 0x92, 0x88, 0x8d, 0xd2, 0x5b, 0x3c, 0xe4, 0xd4, 0x0e, 0x67, 0x30, 0x3c, 0xcd, 0xd6, + 0xe8, 0x06, 0x00, 0x9d, 0xda, 0xf1, 0x64, 0xe8, 0x06, 0xc7, 0xa1, 0xb5, 0x2c, 0xf6, 0xcc, 0x07, + 0x92, 0x33, 0xe2, 0x65, 0xd5, 0x68, 0xba, 0x44, 0x37, 0xa0, 0xe1, 0xdb, 0xa7, 0xae, 0x9f, 0xf8, + 0xbc, 0x62, 0x6e, 0xe0, 0x0c, 0xa9, 0x17, 0x32, 0x6a, 0xad, 0xb4, 0xb4, 0x8d, 0x12, 0x46, 0x8a, + 0x1b, 0x08, 0x6a, 0xc0, 0x99, 0xf6, 0xcf, 0x1a, 0xc0, 0xfc, 0x78, 0x51, 0x1e, 0x46, 0xa2, 0xa1, + 0xef, 0x7a, 0x9e, 0x4b, 0xd5, 0x28, 0x02, 0x87, 0xf6, 0x04, 0x82, 0x5a, 0x50, 0x3a, 0x4e, 0x82, + 0xb1, 0x98, 0xc4, 0xfa, 0x7c, 0x00, 0xee, 0x26, 0xc1, 0x18, 0x0b, 0x06, 0x5d, 0x87, 0xaa, 0x13, + 0x87, 0x49, 0xe4, 0x06, 0x8e, 0x98, 0xa7, 0x7a, 0xcf, 0x4c, 0xbd, 0xee, 0x29, 0x1c, 0x67, 0x1e, + 0xe8, 0x7f, 0x69, 0xb9, 0x0c, 0xe1, 0x9a, 0xa9, 0x01, 0xe6, 0xa0, 0xaa, 0x5e, 0xfb, 0x04, 0x6a, + 0xd9, 0x75, 0x45, 0x8a, 0xaa, 0x2a, 0x13, 0x72, 0x9a, 0xa5, 0x28, 0xf9, 0x09, 0x39, 0x45, 0xff, + 0x85, 0x25, 0x16, 0x32, 0xdb, 0x1b, 0x0a, 0x8c, 0xaa, 0x47, 0x53, 0x17, 0x98, 0x08, 0x43, 0xd1, + 0x0a, 0x14, 0x47, 0x33, 0xf1, 0xfc, 0xab, 0xb8, 0x38, 0x9a, 0x71, 0x99, 0x53, 0xa2, 0x54, 0x6a, + 0xe9, 0x5c, 0xe6, 0xa4, 0xd5, 0x6e, 0x42, 0x89, 0xdf, 0x8c, 0x37, 0x3a, 0xb0, 0xd5, 0xd3, 0xac, + 0x61, 0xb1, 0x6e, 0xf7, 0xa0, 0x9a, 0xde, 0x47, 0xc5, 0xd3, 0xce, 0x89, 0xa7, 0xe7, 0xe2, 0xad, + 0x83, 0x21, 0x2e, 0xc6, 0x1d, 0x72, 0x25, 0x56, 0x56, 0xfb, 0x37, 0x0d, 0x56, 0x52, 0x65, 0x50, + 0x82, 0xb9, 0x01, 0xe5, 0x4c, 0xc1, 0x79, 0x89, 0x56, 0xb2, 0x09, 0x10, 0xe8, 0x4e, 0x01, 0x2b, + 0x1e, 0x35, 0xa1, 0x72, 0x62, 0xc7, 0x01, 0x2f, 0xbc, 0x50, 0xeb, 0x9d, 0x02, 0x4e, 0x01, 0x74, + 0x3d, 0x1d, 0x6b, 0xfd, 0xf5, 0x63, 0xbd, 0x53, 0x48, 0x07, 0xfb, 0x1e, 0x5c, 0x18, 0x87, 0x7e, + 0x14, 0x13, 0x4a, 0xc9, 0x64, 0xa8, 0x8e, 0x97, 0xcd, 0xb4, 0xd2, 0xe3, 0xb7, 0x33, 0x87, 0x2c, + 0x11, 0x73, 0x7c, 0x06, 0xdb, 0xaa, 0x42, 0x39, 0x26, 0x34, 0xf1, 0x58, 0xfb, 0x97, 0x22, 0x5c, + 0x10, 0xa2, 0xb4, 0x6f, 0xfb, 0x73, 0xdd, 0x7b, 0xa3, 0x4e, 0x68, 0xef, 0xa1, 0x13, 0xc5, 0xf7, + 0xd4, 0x89, 0x06, 0x18, 0x94, 0xd9, 0x31, 0x53, 0xbf, 0x11, 0xd2, 0x40, 0x26, 0xe8, 0x24, 0x98, + 0x28, 0x99, 0xe4, 0xcb, 0xb9, 0x5c, 0x18, 0x6f, 0x97, 0x8b, 0x45, 0xb9, 0x2e, 0xff, 0x7b, 0xb9, + 0x6e, 0xc7, 0x80, 0x16, 0x2b, 0xa7, 0xe6, 0xa2, 0x01, 0x06, 0x9f, 0x43, 0xf9, 0x3b, 0x5a, 0xc3, + 0xd2, 0x40, 0x4d, 0xa8, 0xaa, 0x96, 0xf3, 0xc1, 0xe7, 0x44, 0x66, 0xcf, 0x73, 0xd5, 0xdf, 0x9a, + 0x6b, 0xfb, 0xd7, 0xa2, 0x3a, 0xf4, 0x91, 0xed, 0x25, 0xf3, 0x7e, 0x35, 0xc0, 0x10, 0xa3, 0xac, + 0x5e, 0x82, 0x34, 0xde, 0xdc, 0xc5, 0xe2, 0x7b, 0x74, 0x51, 0xff, 0x50, 0x5d, 0x2c, 0x9d, 0xd3, + 0x45, 0xe3, 0x9c, 0x2e, 0x96, 0xdf, 0xad, 0x8b, 0x95, 0x77, 0xe8, 0x62, 0x02, 0x17, 0x73, 0x05, + 0x55, 0x6d, 0xbc, 0x0c, 0xe5, 0x6f, 0x05, 0xa2, 0xfa, 0xa8, 0xac, 0x0f, 0xd5, 0xc8, 0x6b, 0xdf, + 0x40, 0x2d, 0xfb, 0x76, 0x41, 0x75, 0xa8, 0x1c, 0xee, 0x7f, 0xbe, 0xff, 0xe0, 0x68, 0xdf, 0x2c, + 0xa0, 0x1a, 0x18, 0x0f, 0x0f, 0xfb, 0xf8, 0x2b, 0x53, 0x43, 0x55, 0x28, 0xe1, 0xc3, 0xfb, 0x7d, + 0xb3, 0xc8, 0x3d, 0x06, 0xbb, 0x77, 0xfa, 0xdb, 0x9b, 0xd8, 0xd4, 0xb9, 0xc7, 0xe0, 0xe0, 0x01, + 0xee, 0x9b, 0x25, 0x8e, 0xe3, 0xfe, 0x76, 0x7f, 0xf7, 0x51, 0xdf, 0x34, 0x38, 0x7e, 0xa7, 0xbf, + 0x75, 0x78, 0xcf, 0x2c, 0x5f, 0xdb, 0x82, 0x12, 0xff, 0xf1, 0x47, 0x15, 0xd0, 0xf1, 0xe6, 0x91, + 0x8c, 0xba, 0xfd, 0xe0, 0x70, 0xff, 0xc0, 0xd4, 0x38, 0x36, 0x38, 0xdc, 0x33, 0x8b, 0x7c, 0xb1, + 0xb7, 0xbb, 0x6f, 0xea, 0x62, 0xb1, 0xf9, 0xa5, 0x0c, 0x27, 0xbc, 0xfa, 0xd8, 0x34, 0x7a, 0xdf, + 0x15, 0xc1, 0x10, 0x39, 0xa2, 0x8f, 0xa1, 0x24, 0x34, 0xfe, 0x62, 0x5a, 0xd1, 0x85, 0x4f, 0xc9, + 0x66, 0x23, 0x0f, 0xaa, 0xfa, 0x7d, 0x0a, 0x65, 0xa9, 0x35, 0xe8, 0x52, 0x5e, 0x18, 0xd3, 0x6d, + 0x97, 0xcf, 0xc2, 0x72, 0xe3, 0x0d, 0x0d, 0x6d, 0x03, 0xcc, 0xdf, 0x15, 0x5a, 0xcd, 0x75, 0x71, + 0x51, 0xa5, 0x9a, 0xcd, 0xf3, 0x28, 0x75, 0xfe, 0x5d, 0xa8, 0x2f, 0xb4, 0x15, 0xe5, 0x5d, 0x73, + 0x8f, 0xa7, 0x79, 0xf5, 0x5c, 0x4e, 0xc6, 0xe9, 0xed, 0xc3, 0x8a, 0xf8, 0x78, 0xe7, 0xaf, 0x42, + 0x16, 0xe3, 0x36, 0xd4, 0x31, 0xf1, 0x43, 0x46, 0x04, 0x8e, 0xb2, 0xeb, 0x2f, 0x7e, 0xe3, 0x37, + 0x2f, 0x9d, 0x41, 0xd5, 0xff, 0x02, 0x85, 0xad, 0xff, 0x3f, 0xfb, 0x6b, 0xad, 0xf0, 0xec, 0xe5, + 0x9a, 0xf6, 0xfc, 0xe5, 0x9a, 0xf6, 0xe7, 0xcb, 0x35, 0xed, 0xc7, 0x57, 0x6b, 0x85, 0xe7, 0xaf, + 0xd6, 0x0a, 0xbf, 0xbf, 0x5a, 0x2b, 0x3c, 0xae, 0xa8, 0x7f, 0x47, 0x46, 0x65, 0x31, 0x33, 0x37, + 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xdd, 0x80, 0xf5, 0xc0, 0xf8, 0x0c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1412,6 +1433,11 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.MaximumStringSlots != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.MaximumStringSlots)) + i-- + dAtA[i] = 0x70 + } if m.ShardInfo != nil { { size, err := m.ShardInfo.MarshalToSizedBuffer(dAtA[:i]) @@ -1837,6 +1863,27 @@ func (m *SeriesResponse_Hints) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *SeriesResponse_CompressedSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesResponse_CompressedSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.CompressedSeries != nil { + { + size, err := m.CompressedSeries.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} func (m *LabelNamesRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2234,6 +2281,9 @@ func (m *SeriesRequest) Size() (n int) { l = m.ShardInfo.Size() n += 1 + l + sovRpc(uint64(l)) } + if m.MaximumStringSlots != 0 { + n += 1 + sovRpc(uint64(m.MaximumStringSlots)) + } return n } @@ -2374,6 +2424,18 @@ func (m *SeriesResponse_Hints) Size() (n int) { } return n } +func (m *SeriesResponse_CompressedSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CompressedSeries != nil { + l = m.CompressedSeries.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} func (m *LabelNamesRequest) Size() (n int) { if m == nil { return 0 @@ -3300,6 +3362,25 @@ func (m *SeriesRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaximumStringSlots", wireType) + } + m.MaximumStringSlots = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaximumStringSlots |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) @@ -4022,6 +4103,41 @@ func (m *SeriesResponse) Unmarshal(dAtA []byte) error { } m.Result = &SeriesResponse_Hints{v} iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CompressedSeries", 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 := &CompressedSeries{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SeriesResponse_CompressedSeries{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/store/storepb/rpc.proto b/pkg/store/storepb/rpc.proto index 72afaba8ed..74bbc260b2 100644 --- a/pkg/store/storepb/rpc.proto +++ b/pkg/store/storepb/rpc.proto @@ -122,6 +122,12 @@ message SeriesRequest { // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo shard_info = 13; + + // maximum_string_slots is the number of maximum strings + // that the receiver can send. It is safe to ignore this by + // the receiver if it does not intend on sending a symbol table + // via hints later on. + uint64 maximum_string_slots = 14; } @@ -196,6 +202,10 @@ message SeriesResponse { /// 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; + + /// CompressedSeries is like series but instead of raw strings for label names/values, + /// it contains references to strings that are later sent via hints. + CompressedSeries compressed_series = 4; } } diff --git a/pkg/store/storepb/types.pb.go b/pkg/store/storepb/types.pb.go index a87a135ba0..a3267f75f9 100644 --- a/pkg/store/storepb/types.pb.go +++ b/pkg/store/storepb/types.pb.go @@ -11,8 +11,8 @@ import ( _ "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" + labelpb "github.com/thanos-io/thanos/pkg/store/labelpb" ) // Reference imports to suppress errors if they are not otherwise used. @@ -113,7 +113,7 @@ func (x LabelMatcher_Type) String() string { } func (LabelMatcher_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_121fba57de02d8e0, []int{3, 0} + return fileDescriptor_121fba57de02d8e0, []int{4, 0} } type Chunk struct { @@ -155,6 +155,44 @@ func (m *Chunk) XXX_DiscardUnknown() { var xxx_messageInfo_Chunk proto.InternalMessageInfo +type CompressedSeries struct { + Labels []labelpb.CompressedLabel `protobuf:"bytes,1,rep,name=labels,proto3" json:"labels"` + Chunks []AggrChunk `protobuf:"bytes,2,rep,name=chunks,proto3" json:"chunks"` +} + +func (m *CompressedSeries) Reset() { *m = CompressedSeries{} } +func (m *CompressedSeries) String() string { return proto.CompactTextString(m) } +func (*CompressedSeries) ProtoMessage() {} +func (*CompressedSeries) Descriptor() ([]byte, []int) { + return fileDescriptor_121fba57de02d8e0, []int{1} +} +func (m *CompressedSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CompressedSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CompressedSeries.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 *CompressedSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompressedSeries.Merge(m, src) +} +func (m *CompressedSeries) XXX_Size() int { + return m.Size() +} +func (m *CompressedSeries) XXX_DiscardUnknown() { + xxx_messageInfo_CompressedSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_CompressedSeries 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"` @@ -164,7 +202,7 @@ func (m *Series) Reset() { *m = Series{} } func (m *Series) String() string { return proto.CompactTextString(m) } func (*Series) ProtoMessage() {} func (*Series) Descriptor() ([]byte, []int) { - return fileDescriptor_121fba57de02d8e0, []int{1} + return fileDescriptor_121fba57de02d8e0, []int{2} } func (m *Series) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -208,7 +246,7 @@ func (m *AggrChunk) Reset() { *m = AggrChunk{} } func (m *AggrChunk) String() string { return proto.CompactTextString(m) } func (*AggrChunk) ProtoMessage() {} func (*AggrChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_121fba57de02d8e0, []int{2} + return fileDescriptor_121fba57de02d8e0, []int{3} } func (m *AggrChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -248,7 +286,7 @@ func (m *LabelMatcher) Reset() { *m = LabelMatcher{} } func (m *LabelMatcher) String() string { return proto.CompactTextString(m) } func (*LabelMatcher) ProtoMessage() {} func (*LabelMatcher) Descriptor() ([]byte, []int) { - return fileDescriptor_121fba57de02d8e0, []int{3} + return fileDescriptor_121fba57de02d8e0, []int{4} } func (m *LabelMatcher) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -282,6 +320,7 @@ func init() { proto.RegisterEnum("thanos.Chunk_Encoding", Chunk_Encoding_name, Chunk_Encoding_value) proto.RegisterEnum("thanos.LabelMatcher_Type", LabelMatcher_Type_name, LabelMatcher_Type_value) proto.RegisterType((*Chunk)(nil), "thanos.Chunk") + proto.RegisterType((*CompressedSeries)(nil), "thanos.CompressedSeries") proto.RegisterType((*Series)(nil), "thanos.Series") proto.RegisterType((*AggrChunk)(nil), "thanos.AggrChunk") proto.RegisterType((*LabelMatcher)(nil), "thanos.LabelMatcher") @@ -290,42 +329,44 @@ func init() { func init() { proto.RegisterFile("store/storepb/types.proto", fileDescriptor_121fba57de02d8e0) } var fileDescriptor_121fba57de02d8e0 = []byte{ - // 553 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x93, 0xcf, 0x6e, 0xd3, 0x4e, - 0x10, 0xc7, 0xbd, 0xb6, 0xe3, 0x24, 0xf3, 0x6b, 0x7f, 0x32, 0xab, 0x0a, 0xdc, 0x1e, 0x9c, 0xc8, - 0x08, 0x11, 0x55, 0xaa, 0x2d, 0x15, 0x8e, 0x5c, 0x12, 0x14, 0x01, 0x12, 0x6d, 0xe8, 0x26, 0x12, - 0xa8, 0x17, 0xb4, 0x71, 0x57, 0xb6, 0xd5, 0xf8, 0x8f, 0xbc, 0x6b, 0x48, 0x1e, 0x80, 0x3b, 0x88, - 0x3b, 0xcf, 0x93, 0x63, 0x8f, 0x88, 0x43, 0x04, 0xc9, 0x8b, 0x20, 0xaf, 0x1d, 0x20, 0x52, 0x2e, - 0xd6, 0x78, 0x3e, 0xdf, 0x99, 0xd9, 0x99, 0x9d, 0x85, 0x63, 0x2e, 0xd2, 0x9c, 0x79, 0xf2, 0x9b, - 0x4d, 0x3d, 0xb1, 0xc8, 0x18, 0x77, 0xb3, 0x3c, 0x15, 0x29, 0x36, 0x44, 0x48, 0x93, 0x94, 0x9f, - 0x1c, 0x05, 0x69, 0x90, 0x4a, 0x97, 0x57, 0x5a, 0x15, 0x3d, 0xa9, 0x03, 0x67, 0x74, 0xca, 0x66, - 0xbb, 0x81, 0xce, 0x27, 0x04, 0x8d, 0xe7, 0x61, 0x91, 0xdc, 0xe2, 0x53, 0xd0, 0x4b, 0x60, 0xa1, - 0x2e, 0xea, 0xfd, 0x7f, 0x7e, 0xdf, 0xad, 0x32, 0xba, 0x12, 0xba, 0xc3, 0xc4, 0x4f, 0x6f, 0xa2, - 0x24, 0x20, 0x52, 0x83, 0x31, 0xe8, 0x37, 0x54, 0x50, 0x4b, 0xed, 0xa2, 0xde, 0x01, 0x91, 0x36, - 0xb6, 0x40, 0x0f, 0x29, 0x0f, 0x2d, 0xad, 0x8b, 0x7a, 0xfa, 0x40, 0x5f, 0xae, 0x3a, 0x88, 0x48, - 0x8f, 0xe3, 0x40, 0x6b, 0x1b, 0x8f, 0x9b, 0xa0, 0xbd, 0x1b, 0x11, 0x53, 0xc1, 0x87, 0xd0, 0x7e, - 0xf9, 0x6a, 0x3c, 0x19, 0xbd, 0x20, 0xfd, 0x0b, 0x13, 0x39, 0xdf, 0x10, 0x18, 0x63, 0x96, 0x47, - 0x8c, 0x63, 0x1f, 0x0c, 0x79, 0x52, 0x6e, 0xa1, 0xae, 0xd6, 0xfb, 0xef, 0xfc, 0x70, 0x7b, 0x94, - 0xd7, 0xa5, 0x77, 0xf0, 0x6c, 0xb9, 0xea, 0x28, 0x3f, 0x56, 0x9d, 0xa7, 0x41, 0x24, 0xc2, 0x62, - 0xea, 0xfa, 0x69, 0xec, 0x55, 0x82, 0xb3, 0x28, 0xad, 0x2d, 0x2f, 0xbb, 0x0d, 0xbc, 0x9d, 0xa6, - 0xdd, 0x6b, 0x19, 0x4d, 0xea, 0xd4, 0xd8, 0x03, 0xc3, 0x2f, 0x3b, 0xe3, 0x96, 0x2a, 0x8b, 0xdc, - 0xdb, 0x16, 0xe9, 0x07, 0x41, 0x2e, 0x7b, 0x96, 0x2d, 0x28, 0xa4, 0x96, 0x39, 0x5f, 0x55, 0x68, - 0xff, 0x61, 0xf8, 0x18, 0x5a, 0x71, 0x94, 0xbc, 0x17, 0x51, 0x5c, 0x0d, 0x4c, 0x23, 0xcd, 0x38, - 0x4a, 0x26, 0x51, 0xcc, 0x24, 0xa2, 0xf3, 0x0a, 0xa9, 0x35, 0xa2, 0x73, 0x89, 0x3a, 0xa0, 0xe5, - 0xf4, 0xa3, 0x9c, 0xd0, 0x3f, 0x6d, 0xc9, 0x8c, 0xa4, 0x24, 0xf8, 0x21, 0x34, 0xfc, 0xb4, 0x48, - 0x84, 0xa5, 0xef, 0x93, 0x54, 0xac, 0xcc, 0xc2, 0x8b, 0xd8, 0x6a, 0xec, 0xcd, 0xc2, 0x8b, 0xb8, - 0x14, 0xc4, 0x51, 0x62, 0x19, 0x7b, 0x05, 0x71, 0x94, 0x48, 0x01, 0x9d, 0x5b, 0xcd, 0xfd, 0x02, - 0x3a, 0xc7, 0x8f, 0xa1, 0x29, 0x6b, 0xb1, 0xdc, 0x6a, 0xed, 0x13, 0x6d, 0xa9, 0xf3, 0x05, 0xc1, - 0x81, 0x1c, 0xec, 0x05, 0x15, 0x7e, 0xc8, 0x72, 0x7c, 0xb6, 0xb3, 0x45, 0xc7, 0x3b, 0x57, 0x57, - 0x6b, 0xdc, 0xc9, 0x22, 0x63, 0x7f, 0x17, 0x29, 0xa1, 0xf5, 0xa0, 0xda, 0x44, 0xda, 0xf8, 0x08, - 0x1a, 0x1f, 0xe8, 0xac, 0x60, 0x72, 0x4e, 0x6d, 0x52, 0xfd, 0x38, 0x3d, 0xd0, 0xcb, 0x38, 0x6c, - 0x80, 0x3a, 0xbc, 0x32, 0x95, 0x72, 0x91, 0x2e, 0x87, 0x57, 0x26, 0x2a, 0x1d, 0x64, 0x68, 0xaa, - 0xd2, 0x41, 0x86, 0xa6, 0x76, 0xea, 0xc2, 0x83, 0x37, 0x34, 0x17, 0x11, 0x9d, 0x11, 0xc6, 0xb3, - 0x34, 0xe1, 0x6c, 0x2c, 0x72, 0x2a, 0x58, 0xb0, 0xc0, 0x2d, 0xd0, 0xdf, 0xf6, 0xc9, 0xa5, 0xa9, - 0xe0, 0x36, 0x34, 0xfa, 0x83, 0x11, 0x99, 0x98, 0x68, 0xf0, 0x68, 0xf9, 0xcb, 0x56, 0x96, 0x6b, - 0x1b, 0xdd, 0xad, 0x6d, 0xf4, 0x73, 0x6d, 0xa3, 0xcf, 0x1b, 0x5b, 0xb9, 0xdb, 0xd8, 0xca, 0xf7, - 0x8d, 0xad, 0x5c, 0x37, 0xeb, 0xe7, 0x36, 0x35, 0xe4, 0x83, 0x79, 0xf2, 0x3b, 0x00, 0x00, 0xff, - 0xff, 0x44, 0x9d, 0x95, 0xa2, 0x86, 0x03, 0x00, 0x00, + // 577 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0x31, 0x6f, 0xd3, 0x40, + 0x14, 0xc7, 0x7d, 0xb6, 0xe3, 0x24, 0x8f, 0x16, 0x99, 0x53, 0x45, 0xdd, 0x0e, 0x6e, 0x64, 0x84, + 0x88, 0x2a, 0xd5, 0x96, 0x0a, 0x6c, 0x2c, 0x69, 0x15, 0x01, 0x12, 0x6d, 0xe9, 0x35, 0x12, 0xa8, + 0x0b, 0xba, 0xb8, 0x27, 0xdb, 0x6a, 0xec, 0xb3, 0x7c, 0x67, 0x48, 0xd9, 0xd9, 0x41, 0xec, 0x7c, + 0x9e, 0x8e, 0x1d, 0x11, 0x43, 0x05, 0xed, 0x17, 0x41, 0x3e, 0x3b, 0x81, 0xa0, 0x4c, 0x2c, 0xd1, + 0xcb, 0xfb, 0xff, 0xfe, 0xef, 0x3d, 0xbf, 0xbb, 0x83, 0x0d, 0x21, 0x79, 0xc1, 0x02, 0xf5, 0x9b, + 0x8f, 0x03, 0x79, 0x91, 0x33, 0xe1, 0xe7, 0x05, 0x97, 0x1c, 0x5b, 0x32, 0xa6, 0x19, 0x17, 0x9b, + 0x6b, 0x11, 0x8f, 0xb8, 0x4a, 0x05, 0x55, 0x54, 0xab, 0x9b, 0x8d, 0x71, 0x42, 0xc7, 0x6c, 0xb2, + 0x68, 0xf4, 0x3e, 0x21, 0x68, 0xed, 0xc7, 0x65, 0x76, 0x8e, 0xb7, 0xc1, 0xac, 0x04, 0x07, 0xf5, + 0x50, 0xff, 0xee, 0xee, 0x7d, 0xbf, 0xae, 0xe8, 0x2b, 0xd1, 0x1f, 0x66, 0x21, 0x3f, 0x4b, 0xb2, + 0x88, 0x28, 0x06, 0x63, 0x30, 0xcf, 0xa8, 0xa4, 0x8e, 0xde, 0x43, 0xfd, 0x15, 0xa2, 0x62, 0xec, + 0x80, 0x19, 0x53, 0x11, 0x3b, 0x46, 0x0f, 0xf5, 0xcd, 0x3d, 0xf3, 0xf2, 0x7a, 0x0b, 0x11, 0x95, + 0xf1, 0x3c, 0xe8, 0xcc, 0xfc, 0xb8, 0x0d, 0xc6, 0xdb, 0x23, 0x62, 0x6b, 0x78, 0x15, 0xba, 0x2f, + 0x5e, 0x9e, 0x8c, 0x8e, 0x9e, 0x93, 0xc1, 0x81, 0x8d, 0xbc, 0x8f, 0x60, 0xef, 0xf3, 0x34, 0x2f, + 0x98, 0x10, 0xec, 0xec, 0x84, 0x15, 0x09, 0x13, 0xf8, 0x29, 0x58, 0x6a, 0x64, 0xe1, 0xa0, 0x9e, + 0xd1, 0xbf, 0xb3, 0xbb, 0x3e, 0x9f, 0x69, 0x4e, 0xbe, 0xaa, 0x74, 0xd5, 0x4c, 0x23, 0x0d, 0x8c, + 0x03, 0xb0, 0xc2, 0x6a, 0x68, 0xe1, 0xe8, 0xca, 0x76, 0x6f, 0x66, 0x1b, 0x44, 0x51, 0xa1, 0x3e, + 0x67, 0x66, 0xa8, 0x31, 0xef, 0x1b, 0x02, 0xab, 0x69, 0x19, 0xfe, 0xd3, 0x72, 0x75, 0xe6, 0xad, + 0x1b, 0x3d, 0xab, 0x7c, 0x3f, 0xae, 0xb7, 0x9e, 0x44, 0x89, 0x8c, 0xcb, 0xb1, 0x1f, 0xf2, 0x34, + 0xa8, 0x81, 0x9d, 0x84, 0x37, 0x51, 0x90, 0x9f, 0x47, 0xc1, 0xc2, 0xc2, 0xfd, 0x53, 0xe5, 0xfe, + 0xff, 0x01, 0xbf, 0xea, 0xd0, 0x9d, 0x6b, 0x78, 0x03, 0x3a, 0x69, 0x92, 0xbd, 0x93, 0x49, 0x5a, + 0x1f, 0x96, 0x41, 0xda, 0x69, 0x92, 0x8d, 0x92, 0x94, 0x29, 0x89, 0x4e, 0x6b, 0x49, 0x6f, 0x24, + 0x3a, 0x55, 0xd2, 0x16, 0x18, 0x05, 0xfd, 0xa0, 0x4e, 0xe7, 0xaf, 0xcf, 0x52, 0x15, 0x49, 0xa5, + 0xe0, 0x07, 0xd0, 0x0a, 0x79, 0x99, 0x49, 0xc7, 0x5c, 0x86, 0xd4, 0x5a, 0x55, 0x45, 0x94, 0xa9, + 0xd3, 0x5a, 0x5a, 0x45, 0x94, 0x69, 0x05, 0xa4, 0x49, 0xe6, 0x58, 0x4b, 0x81, 0x34, 0xc9, 0x14, + 0x40, 0xa7, 0x4e, 0x7b, 0x39, 0x40, 0xa7, 0xf8, 0x11, 0xb4, 0x55, 0x2f, 0x56, 0x38, 0x9d, 0x65, + 0xd0, 0x4c, 0xf5, 0xbe, 0x20, 0x58, 0x51, 0x8b, 0x3d, 0xa0, 0x32, 0x8c, 0x59, 0x81, 0x77, 0x16, + 0x6e, 0xf0, 0xc6, 0xc2, 0xd1, 0x35, 0x8c, 0x3f, 0xba, 0xc8, 0xd9, 0x9f, 0x4b, 0x9c, 0xd1, 0x66, + 0x51, 0x5d, 0xa2, 0x62, 0xbc, 0x06, 0xad, 0xf7, 0x74, 0x52, 0x32, 0xb5, 0xa7, 0x2e, 0xa9, 0xff, + 0x78, 0x7d, 0x30, 0x2b, 0x1f, 0xb6, 0x40, 0x1f, 0x1e, 0xdb, 0x5a, 0x75, 0x89, 0x0f, 0x87, 0xc7, + 0x36, 0xaa, 0x12, 0x64, 0x68, 0xeb, 0x2a, 0x41, 0x86, 0xb6, 0xb1, 0xed, 0xc3, 0xfa, 0x6b, 0x5a, + 0xc8, 0x84, 0x4e, 0x08, 0x13, 0x39, 0xcf, 0x04, 0x3b, 0x91, 0x05, 0x95, 0x2c, 0xba, 0xc0, 0x1d, + 0x30, 0xdf, 0x0c, 0xc8, 0xa1, 0xad, 0xe1, 0x2e, 0xb4, 0x06, 0x7b, 0x47, 0x64, 0x64, 0xa3, 0xbd, + 0x87, 0x97, 0xbf, 0x5c, 0xed, 0xf2, 0xc6, 0x45, 0x57, 0x37, 0x2e, 0xfa, 0x79, 0xe3, 0xa2, 0xcf, + 0xb7, 0xae, 0x76, 0x75, 0xeb, 0x6a, 0xdf, 0x6f, 0x5d, 0xed, 0xb4, 0xdd, 0x3c, 0xf5, 0xb1, 0xa5, + 0x1e, 0xeb, 0xe3, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xa6, 0xd8, 0xf0, 0xfd, 0x02, 0x04, 0x00, + 0x00, } func (m *Chunk) Marshal() (dAtA []byte, err error) { @@ -368,6 +409,57 @@ func (m *Chunk) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *CompressedSeries) 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 *CompressedSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CompressedSeries) 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, err := m.Labels[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *Series) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -596,6 +688,27 @@ func (m *Chunk) Size() (n int) { return n } +func (m *CompressedSeries) 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 *Series) Size() (n int) { if m == nil { return 0 @@ -804,6 +917,124 @@ func (m *Chunk) Unmarshal(dAtA []byte) error { } return nil } +func (m *CompressedSeries) 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: CompressedSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CompressedSeries: 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, labelpb.CompressedLabel{}) + 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) || (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 diff --git a/pkg/store/storepb/types.proto b/pkg/store/storepb/types.proto index 67c93fa52e..a54a336c98 100644 --- a/pkg/store/storepb/types.proto +++ b/pkg/store/storepb/types.proto @@ -30,6 +30,11 @@ message Chunk { uint64 hash = 3 [(gogoproto.nullable) = true]; } +message CompressedSeries { + repeated CompressedLabel labels = 1 [(gogoproto.nullable) = false]; + repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; +} + message Series { repeated Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/store/symbol_table.go b/pkg/store/symbol_table.go new file mode 100644 index 0000000000..6c5d9beb8c --- /dev/null +++ b/pkg/store/symbol_table.go @@ -0,0 +1,75 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package store + +import ( + "math" + "strings" +) + +type adjusterFn func(uint64) uint64 + +func maxStringsPerStore(storeCount uint64) uint64 { + return math.MaxUint64 / uint64(storeCount) +} + +func newReferenceAdjusterFactory(storeCount uint64) func(storeIndex uint64) adjusterFn { + // Adjuster adjusts each incoming reference according to the number of stores. + // Whole label space is stored in uint64 so that's how many + // strings we are able to store. + eachStore := maxStringsPerStore(storeCount) + + return func(storeIndex uint64) adjusterFn { + startFrom := eachStore * storeIndex + + mappings := map[uint64]uint64{} + currentMappingInd := startFrom + + return func(ref uint64) uint64 { + if i, ok := mappings[ref]; ok { + return i + } + mappings[ref] = currentMappingInd + currentMappingInd++ + return currentMappingInd - 1 + } + } +} + +// symbolTableBuilder provides a way of building +// a symbol table for static strings to compress +// responses better. gRPC compression works on a +// message-by-message basis but we want to compress +// static strings as well. +// It's not safe to use this concurrently. +type symbolTableBuilder struct { + maxElements uint64 + + current uint64 + table map[uint64]string + reverseTable map[string]uint64 +} + +func newSymbolTableBuilder(maxElements uint64) *symbolTableBuilder { + return &symbolTableBuilder{maxElements: maxElements, table: make(map[uint64]string), reverseTable: make(map[string]uint64)} +} + +func (b *symbolTableBuilder) getOrStoreString(s string) (uint64, bool) { + if num, ok := b.reverseTable[s]; ok { + return num, true + } + if b.current >= b.maxElements { + return 0, false + } + + s = strings.Clone(s) + b.reverseTable[s] = b.current + b.table[b.current] = s + b.current++ + return b.current - 1, true +} + +func (b *symbolTableBuilder) getTable() map[uint64]string { + return b.table +} diff --git a/pkg/store/symbol_table_test.go b/pkg/store/symbol_table_test.go new file mode 100644 index 0000000000..b041148caf --- /dev/null +++ b/pkg/store/symbol_table_test.go @@ -0,0 +1,18 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package store + +import ( + "math" + "testing" + + "github.com/efficientgo/core/testutil" +) + +func TestSymbolTableHandlesBigNumbersCorrectly(t *testing.T) { + adjuster := newReferenceAdjusterFactory(2) + adjust := adjuster(0) + + testutil.Assert(t, adjust(0) != adjust(math.MaxUint64/2)) +} diff --git a/test/e2e/query_test.go b/test/e2e/query_test.go index 1c599413ac..f3b6482b76 100644 --- a/test/e2e/query_test.go +++ b/test/e2e/query_test.go @@ -647,6 +647,7 @@ func TestQueryStoreMetrics(t *testing.T) { querier := e2ethanos.NewQuerierBuilder(e, "1", storeGW.InternalEndpoint("grpc")).Init() testutil.Ok(t, e2e.StartAndWaitReady(storeGW, querier)) testutil.Ok(t, storeGW.WaitSumMetrics(e2emon.Equals(2), "thanos_blocks_meta_synced")) + testutil.Ok(t, querier.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) // Querying the series in the previously created blocks to ensure we produce Store API query metrics. { @@ -1704,5 +1705,62 @@ func TestConnectedQueriesWithLazyProxy(t *testing.T) { instantQuery(t, context.Background(), querier2.Endpoint("http"), func() string { return "sum(metric_that_does_not_exist)" }, time.Now, promclient.QueryOptions{}, 0) +} + +// TestCompressionCompatibility tests whether symbol (string) table +// compression is compatible with older versions. +func TestCompressionCompatibility(t *testing.T) { + t.Parallel() + + e, err := e2e.NewDockerEnvironment("compr-compat") + testutil.Ok(t, err) + t.Cleanup(e2ethanos.CleanScenario(t, e)) + + promConfig := e2ethanos.DefaultPromConfig("p1", 0, "", "", e2ethanos.LocalPrometheusTarget) + prom, sidecar := e2ethanos.NewPrometheusWithSidecar(e, "p1", promConfig, "", e2ethanos.DefaultPrometheusImage(), "") + querier1 := e2ethanos.NewQuerierBuilder(e, "1", sidecar.InternalEndpoint("grpc")).WithDisablePartialResponses(true).Init() + querier2 := e2ethanos.NewQuerierBuilder(e, "2", sidecar.InternalEndpoint("grpc")).WithDisablePartialResponses(true).WithImage("quay.io/thanos/thanos:v0.26.0").Init() + querierConnecting := e2ethanos.NewQuerierBuilder(e, "3", querier2.InternalEndpoint("grpc")).WithDisablePartialResponses(true).Init() + testutil.Ok(t, e2e.StartAndWaitReady(prom, sidecar, querier1, querier2, querierConnecting)) + + testutil.Ok(t, querier2.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, querier1.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, querierConnecting.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + + for _, querier := range []*e2emon.InstrumentedRunnable{querier1, querier2, querierConnecting} { + result := instantQuery(t, context.Background(), querier.Endpoint("http"), func() string { + return "sum(up)" + }, time.Now, promclient.QueryOptions{}, 1) + testutil.Equals(t, model.SampleValue(1.0), result[0].Value) + } + +} + +// TestCompressionReferenceAdjustments tests whether symbol (string) table +// compression adjusts references properly in the case of a Querier +// connected to a Querier. +func TestCompressionReferenceAdjustments(t *testing.T) { + t.Parallel() + + e, err := e2e.NewDockerEnvironment("compr-adjust") + testutil.Ok(t, err) + t.Cleanup(e2ethanos.CleanScenario(t, e)) + + promConfig := e2ethanos.DefaultPromConfig("p1", 0, "", "", e2ethanos.LocalPrometheusTarget) + prom, sidecar := e2ethanos.NewPrometheusWithSidecar(e, "p1", promConfig, "", e2ethanos.DefaultPrometheusImage(), "") + + querier1 := e2ethanos.NewQuerierBuilder(e, "1", sidecar.InternalEndpoint("grpc")).WithDisablePartialResponses(true).Init() + querier2 := e2ethanos.NewQuerierBuilder(e, "2", sidecar.InternalEndpoint("grpc")).WithDisablePartialResponses(true).Init() + querierGlobal := e2ethanos.NewQuerierBuilder(e, "3", querier1.InternalEndpoint("grpc"), querier2.InternalEndpoint("grpc")).WithDisablePartialResponses(true).Init() + testutil.Ok(t, e2e.StartAndWaitReady(prom, sidecar, querier1, querier2, querierGlobal)) + + testutil.Ok(t, querier2.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, querier1.WaitSumMetricsWithOptions(e2emon.Equals(1), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, querierGlobal.WaitSumMetricsWithOptions(e2emon.Equals(2), []string{"thanos_store_nodes_grpc_connections"}, e2emon.WaitMissingMetrics())) + + result := instantQuery(t, context.Background(), querierGlobal.Endpoint("http"), func() string { + return "up" + }, time.Now, promclient.QueryOptions{}, 1) + testutil.Equals(t, `up{instance="localhost:9090", job="myself", prometheus="p1", replica="0"}`, result[0].Metric.String()) } diff --git a/test/e2e/store_gateway_test.go b/test/e2e/store_gateway_test.go index 0b16f58bbd..cabc62fbb1 100644 --- a/test/e2e/store_gateway_test.go +++ b/test/e2e/store_gateway_test.go @@ -741,6 +741,11 @@ func TestStoreGatewayBytesLimit(t *testing.T) { m := e2edb.NewMinio(e, "thanos-minio", bucket, e2edb.WithMinioTLS()) testutil.Ok(t, e2e.StartAndWaitReady(m)) + const cacheCfg = `type: IN-MEMORY +config: + max_size: 2B + max_item_size: 1B` + store1 := e2ethanos.NewStoreGW( e, "1", @@ -748,7 +753,7 @@ func TestStoreGatewayBytesLimit(t *testing.T) { Type: client.S3, Config: e2ethanos.NewS3Config(bucket, m.InternalEndpoint("http"), m.InternalDir()), }, - "", + string(cacheCfg), []string{"--store.grpc.downloaded-bytes-limit=1B"}, ) @@ -759,7 +764,7 @@ func TestStoreGatewayBytesLimit(t *testing.T) { Type: client.S3, Config: e2ethanos.NewS3Config(bucket, m.InternalEndpoint("http"), m.InternalDir()), }, - "", + string(cacheCfg), []string{"--store.grpc.downloaded-bytes-limit=100B"}, ) store3 := e2ethanos.NewStoreGW( @@ -769,7 +774,7 @@ func TestStoreGatewayBytesLimit(t *testing.T) { Type: client.S3, Config: e2ethanos.NewS3Config(bucket, m.InternalEndpoint("http"), m.InternalDir()), }, - "", + string(cacheCfg), []string{"--store.grpc.downloaded-bytes-limit=196627B"}, )