diff --git a/CHANGELOG.md b/CHANGELOG.md index 9bf5aa41269..c7c8928b69c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * [CHANGE] Store-gateway: skip verifying index header integrity upon loading. To enable verification set `blocks_storage.bucket_store.index_header.verify_on_load: true`. * [CHANGE] Querier: change the default value of the experimental `-querier.streaming-chunks-per-ingester-buffer-size` flag to 256. #5203 * [CHANGE] Querier: only initiate query requests to ingesters in the `ACTIVE` state in the ring. #5342 +* [CHANGE] Querier: Renamed `-querier.prefer-streaming-chunks` to `-querier.prefer-streaming-chunks-from-ingesters` to enable streaming chunks from ingesters to queriers. #5182 * [CHANGE] Querier: `-query-frontend.cache-unaligned-requests` has been moved from a global flag to a per-tenant override. #5312 * [CHANGE] Ingester: removed `cortex_ingester_shipper_dir_syncs_total` and `cortex_ingester_shipper_dir_sync_failures_total` metrics. The former metric was not much useful, and the latter was never incremented. #5396 * [FEATURE] Cardinality API: Add a new `count_method` parameter which enables counting active series #5136 @@ -42,6 +43,7 @@ * [ENHANCEMENT] Querier: add `cortex_querier_queries_rejected_total` metric that counts the number of queries rejected due to hitting a limit (eg. max series per query or max chunks per query). #5316 #5440 * [ENHANCEMENT] Querier: add experimental `-querier.minimize-ingester-requests-hedging-delay` option to initiate requests to further ingesters when request minimisation is enabled and not all initial requests have completed. #5368 * [ENHANCEMENT] Clarify docs for `-ingester.client.*` flags to make it clear that these are used by both queriers and distributors. #5375 +* [ENHANCEMENT] Querier and store-gateway: add experimental support for streaming chunks from store-gateways to queriers while evaluating queries. This can be enabled with `-querier.prefer-streaming-chunks-from-store-gateways=true`. #5182 * [ENHANCEMENT] Querier: enforce `max-chunks-per-query` limit earlier in query processing when streaming chunks from ingesters to queriers to avoid unnecessarily consuming resources for queries that will be aborted. #5369 #5447 * [ENHANCEMENT] Ingester: added `cortex_ingester_shipper_last_successful_upload_timestamp_seconds` metric tracking the last successful TSDB block uploaded to the bucket (unix timestamp in seconds). #5396 * [BUGFIX] Ingester: Handle when previous ring state is leaving and the number of tokens has changed. #5204 diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index acc2023782b..6683944032b 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -1642,12 +1642,23 @@ }, { "kind": "field", - "name": "prefer_streaming_chunks", + "name": "prefer_streaming_chunks_from_ingesters", "required": false, "desc": "Request ingesters stream chunks. Ingesters will only respond with a stream of chunks if the target ingester supports this, and this preference will be ignored by ingesters that do not support this.", "fieldValue": null, "fieldDefaultValue": false, - "fieldFlag": "querier.prefer-streaming-chunks", + "fieldFlag": "querier.prefer-streaming-chunks-from-ingesters", + "fieldType": "boolean", + "fieldCategory": "experimental" + }, + { + "kind": "field", + "name": "prefer_streaming_chunks_from_store_gateways", + "required": false, + "desc": "Request store-gateways stream chunks. Store-gateways will only respond with a stream of chunks if the target store-gateway supports this, and this preference will be ignored by store-gateways that do not support this.", + "fieldValue": null, + "fieldDefaultValue": false, + "fieldFlag": "querier.prefer-streaming-chunks-from-store-gateways", "fieldType": "boolean", "fieldCategory": "experimental" }, @@ -1662,6 +1673,17 @@ "fieldType": "int", "fieldCategory": "experimental" }, + { + "kind": "field", + "name": "streaming_chunks_per_store_gateway_series_buffer_size", + "required": false, + "desc": "Number of series to buffer per store-gateway when streaming chunks from store-gateways.", + "fieldValue": null, + "fieldDefaultValue": 256, + "fieldFlag": "querier.streaming-chunks-per-store-gateway-buffer-size", + "fieldType": "int", + "fieldCategory": "experimental" + }, { "kind": "field", "name": "minimize_ingester_requests", diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index a5ad2d5ddd2..28e0ac71c47 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -1611,8 +1611,10 @@ Usage of ./cmd/mimir/mimir: [experimental] If true, when querying ingesters, only the minimum required ingesters required to reach quorum will be queried initially, with other ingesters queried only if needed due to failures from the initial set of ingesters. Enabling this option reduces resource consumption for the happy path at the cost of increased latency for the unhappy path. -querier.minimize-ingester-requests-hedging-delay duration [experimental] Delay before initiating requests to further ingesters when request minimization is enabled and the initially selected set of ingesters have not all responded. Ignored if -querier.minimize-ingester-requests is not enabled. (default 3s) - -querier.prefer-streaming-chunks + -querier.prefer-streaming-chunks-from-ingesters [experimental] Request ingesters stream chunks. Ingesters will only respond with a stream of chunks if the target ingester supports this, and this preference will be ignored by ingesters that do not support this. + -querier.prefer-streaming-chunks-from-store-gateways + [experimental] Request store-gateways stream chunks. Store-gateways will only respond with a stream of chunks if the target store-gateway supports this, and this preference will be ignored by store-gateways that do not support this. -querier.query-ingesters-within duration Maximum lookback beyond which queries are not sent to ingester. 0 means all queries are sent to ingester. (default 13h) -querier.query-store-after duration @@ -1639,6 +1641,8 @@ Usage of ./cmd/mimir/mimir: Override the expected name on the server certificate. -querier.streaming-chunks-per-ingester-buffer-size uint [experimental] Number of series to buffer per ingester when streaming chunks from ingesters. (default 256) + -querier.streaming-chunks-per-store-gateway-buffer-size uint + [experimental] Number of series to buffer per store-gateway when streaming chunks from store-gateways. (default 256) -querier.timeout duration The timeout for a query. This config option should be set on query-frontend too when query sharding is enabled. This also applies to queries evaluated by the ruler (internally or remotely). (default 2m0s) -query-frontend.align-queries-with-step diff --git a/docs/sources/mimir/references/configuration-parameters/index.md b/docs/sources/mimir/references/configuration-parameters/index.md index 465330c9637..cc65d9beb4c 100644 --- a/docs/sources/mimir/references/configuration-parameters/index.md +++ b/docs/sources/mimir/references/configuration-parameters/index.md @@ -1096,14 +1096,25 @@ store_gateway_client: # (experimental) Request ingesters stream chunks. Ingesters will only respond # with a stream of chunks if the target ingester supports this, and this # preference will be ignored by ingesters that do not support this. -# CLI flag: -querier.prefer-streaming-chunks -[prefer_streaming_chunks: | default = false] +# CLI flag: -querier.prefer-streaming-chunks-from-ingesters +[prefer_streaming_chunks_from_ingesters: | default = false] + +# (experimental) Request store-gateways stream chunks. Store-gateways will only +# respond with a stream of chunks if the target store-gateway supports this, and +# this preference will be ignored by store-gateways that do not support this. +# CLI flag: -querier.prefer-streaming-chunks-from-store-gateways +[prefer_streaming_chunks_from_store_gateways: | default = false] # (experimental) Number of series to buffer per ingester when streaming chunks # from ingesters. # CLI flag: -querier.streaming-chunks-per-ingester-buffer-size [streaming_chunks_per_ingester_series_buffer_size: | default = 256] +# (experimental) Number of series to buffer per store-gateway when streaming +# chunks from store-gateways. +# CLI flag: -querier.streaming-chunks-per-store-gateway-buffer-size +[streaming_chunks_per_store_gateway_series_buffer_size: | default = 256] + # (experimental) If true, when querying ingesters, only the minimum required # ingesters required to reach quorum will be queried initially, with other # ingesters queried only if needed due to failures from the initial set of diff --git a/integration/ingester_test.go b/integration/ingester_test.go index 88dc4c5a678..d73fc1c523f 100644 --- a/integration/ingester_test.go +++ b/integration/ingester_test.go @@ -480,9 +480,9 @@ func TestIngesterQuerying(t *testing.T) { defer s.Close() baseFlags := map[string]string{ - "-distributor.ingestion-tenant-shard-size": "0", - "-ingester.ring.heartbeat-period": "1s", - "-querier.prefer-streaming-chunks": strconv.FormatBool(streamingEnabled), + "-distributor.ingestion-tenant-shard-size": "0", + "-ingester.ring.heartbeat-period": "1s", + "-querier.prefer-streaming-chunks-from-ingesters": strconv.FormatBool(streamingEnabled), } flags := mergeFlags( @@ -543,12 +543,12 @@ func TestIngesterQueryingWithRequestMinimization(t *testing.T) { defer s.Close() baseFlags := map[string]string{ - "-distributor.ingestion-tenant-shard-size": "0", - "-ingester.ring.heartbeat-period": "1s", - "-ingester.ring.zone-awareness-enabled": "true", - "-ingester.ring.replication-factor": "3", - "-querier.minimize-ingester-requests": "true", - "-querier.prefer-streaming-chunks": strconv.FormatBool(streamingEnabled), + "-distributor.ingestion-tenant-shard-size": "0", + "-ingester.ring.heartbeat-period": "1s", + "-ingester.ring.zone-awareness-enabled": "true", + "-ingester.ring.replication-factor": "3", + "-querier.minimize-ingester-requests": "true", + "-querier.prefer-streaming-chunks-from-ingesters": strconv.FormatBool(streamingEnabled), } flags := mergeFlags( diff --git a/integration/querier_test.go b/integration/querier_test.go index e447b45a657..463e4071836 100644 --- a/integration/querier_test.go +++ b/integration/querier_test.go @@ -28,14 +28,22 @@ import ( ) func TestQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T) { - testQuerierWithBlocksStorageRunningInMicroservicesMode(t, generateFloatSeries) + for _, streamingEnabled := range []bool{true, false} { + t.Run(fmt.Sprintf("streaming=%t", streamingEnabled), func(t *testing.T) { + testQuerierWithBlocksStorageRunningInMicroservicesMode(t, streamingEnabled, generateFloatSeries) + }) + } } func TestQuerierWithBlocksStorageRunningInMicroservicesModeWithHistograms(t *testing.T) { - testQuerierWithBlocksStorageRunningInMicroservicesMode(t, generateHistogramSeries) + for _, streamingEnabled := range []bool{true, false} { + t.Run(fmt.Sprintf("streaming=%t", streamingEnabled), func(t *testing.T) { + testQuerierWithBlocksStorageRunningInMicroservicesMode(t, streamingEnabled, generateHistogramSeries) + }) + } } -func testQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T, seriesGenerator func(name string, ts time.Time, additionalLabels ...prompb.Label) (series []prompb.TimeSeries, vector model.Vector, matrix model.Matrix)) { +func testQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T, streamingEnabled bool, seriesGenerator func(name string, ts time.Time, additionalLabels ...prompb.Label) (series []prompb.TimeSeries, vector model.Vector, matrix model.Matrix)) { tests := map[string]struct { tenantShardSize int indexCacheBackend string @@ -162,6 +170,8 @@ func testQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T, series "-store-gateway.tenant-shard-size": fmt.Sprintf("%d", testCfg.tenantShardSize), "-query-frontend.query-stats-enabled": "true", "-query-frontend.parallelize-shardable-queries": strconv.FormatBool(testCfg.queryShardingEnabled), + "-querier.prefer-streaming-chunks-from-ingesters": strconv.FormatBool(streamingEnabled), + "-querier.prefer-streaming-chunks-from-store-gateways": strconv.FormatBool(streamingEnabled), }) // Start store-gateways. @@ -239,15 +249,21 @@ func testQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T, series // thanos_store_index_cache_requests_total: ExpandedPostings: 5, Postings: 2, Series: 2 instantQueriesCount++ + comparingFunction := e2e.Equals + if streamingEnabled { + // Some metrics can be higher when streaming is enabled. The exact number is not deterministic in every case. + comparingFunction = e2e.GreaterOrEqual + } + // Check the in-memory index cache metrics (in the store-gateway). - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(9), "thanos_store_index_cache_requests_total")) // 5 + 2 + 2 - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(0), "thanos_store_index_cache_hits_total")) // no cache hit cause the cache was empty + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(9), "thanos_store_index_cache_requests_total")) // 5 + 2 + 2 + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(0), "thanos_store_index_cache_hits_total")) // no cache hit cause the cache was empty if testCfg.indexCacheBackend == tsdb.IndexCacheBackendInMemory { require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(2*2+2+3), "thanos_store_index_cache_items")) // 2 series both for postings and series cache, 2 expanded postings on one block, 3 on another one require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(2*2+2+3), "thanos_store_index_cache_items_added_total")) // 2 series both for postings and series cache, 2 expanded postings on one block, 3 on another one } else if testCfg.indexCacheBackend == tsdb.IndexCacheBackendMemcached { - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(9*2), "thanos_memcached_operations_total")) // one set for each get + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(9*2), "thanos_memcached_operations_total")) // one set for each get } // Query back again the 1st series from storage. This time it should use the index cache. @@ -257,14 +273,14 @@ func testQuerierWithBlocksStorageRunningInMicroservicesMode(t *testing.T, series assert.Equal(t, expectedVector1, result.(model.Vector)) expectedFetchedSeries++ // Storage only. - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(9+2), "thanos_store_index_cache_requests_total")) - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(2), "thanos_store_index_cache_hits_total")) // this time has used the index cache + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(9+2), "thanos_store_index_cache_requests_total")) + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(2), "thanos_store_index_cache_hits_total")) // this time has used the index cache if testCfg.indexCacheBackend == tsdb.IndexCacheBackendInMemory { require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(2*2+2+3), "thanos_store_index_cache_items")) // as before require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(2*2+2+3), "thanos_store_index_cache_items_added_total")) // as before } else if testCfg.indexCacheBackend == tsdb.IndexCacheBackendMemcached { - require.NoError(t, storeGateways.WaitSumMetrics(e2e.Equals(9*2+2), "thanos_memcached_operations_total")) // as before + 2 gets (expanded postings and series) + require.NoError(t, storeGateways.WaitSumMetrics(comparingFunction(9*2+2), "thanos_memcached_operations_total")) // as before + 2 gets (expanded postings and series) } // Query range. We expect 1 data point with a value of 3 (number of series). @@ -877,12 +893,13 @@ func TestQueryLimitsWithBlocksStorageRunningInMicroServices(t *testing.T) { // Configure the blocks storage to frequently compact TSDB head // and ship blocks to the storage. flags := mergeFlags(BlocksStorageFlags(), BlocksStorageS3Flags(), map[string]string{ - "-blocks-storage.tsdb.block-ranges-period": blockRangePeriod.String(), - "-blocks-storage.tsdb.ship-interval": "1s", - "-blocks-storage.bucket-store.sync-interval": "1s", - "-blocks-storage.tsdb.retention-period": ((blockRangePeriod * 2) - 1).String(), - "-querier.max-fetched-series-per-query": "3", - "-querier.prefer-streaming-chunks": strconv.FormatBool(streamingEnabled), + "-blocks-storage.tsdb.block-ranges-period": blockRangePeriod.String(), + "-blocks-storage.tsdb.ship-interval": "1s", + "-blocks-storage.bucket-store.sync-interval": "1s", + "-blocks-storage.tsdb.retention-period": ((blockRangePeriod * 2) - 1).String(), + "-querier.max-fetched-series-per-query": "3", + "-querier.prefer-streaming-chunks-from-ingesters": strconv.FormatBool(streamingEnabled), + "-querier.prefer-streaming-chunks-from-store-gateways": strconv.FormatBool(streamingEnabled), }) // Start dependencies. diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 1c3142aaafe..9be96f18286 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -166,7 +166,7 @@ type Config struct { // This config is dynamically injected because it is defined in the querier config. ShuffleShardingLookbackPeriod time.Duration `yaml:"-"` - PreferStreamingChunks bool `yaml:"-"` + PreferStreamingChunksFromIngesters bool `yaml:"-"` StreamingChunksPerIngesterSeriesBufferSize uint64 `yaml:"-"` MinimizeIngesterRequests bool `yaml:"-"` MinimiseIngesterRequestsHedgingDelay time.Duration `yaml:"-"` diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index bc6e529062c..7e09ca73f42 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -3023,7 +3023,7 @@ func prepare(t *testing.T, cfg prepConfig) ([]*Distributor, []mockIngester, []*p distributorCfg.DefaultLimits.MaxInflightPushRequestsBytes = cfg.maxInflightRequestsBytes distributorCfg.DefaultLimits.MaxIngestionRate = cfg.maxIngestionRate distributorCfg.ShuffleShardingLookbackPeriod = time.Hour - distributorCfg.PreferStreamingChunks = cfg.preferStreamingChunks + distributorCfg.PreferStreamingChunksFromIngesters = cfg.preferStreamingChunks distributorCfg.StreamingChunksPerIngesterSeriesBufferSize = 128 cfg.limits.IngestionTenantShardSize = cfg.shuffleShardSize diff --git a/pkg/distributor/query.go b/pkg/distributor/query.go index a72cccb732b..c7d70897164 100644 --- a/pkg/distributor/query.go +++ b/pkg/distributor/query.go @@ -71,7 +71,7 @@ func (d *Distributor) QueryStream(ctx context.Context, from, to model.Time, matc return err } - if d.cfg.PreferStreamingChunks { + if d.cfg.PreferStreamingChunksFromIngesters { req.StreamingChunksBatchSize = d.cfg.StreamingChunksPerIngesterSeriesBufferSize } diff --git a/pkg/mimir/modules.go b/pkg/mimir/modules.go index 57840522e87..ae7765410c9 100644 --- a/pkg/mimir/modules.go +++ b/pkg/mimir/modules.go @@ -361,7 +361,7 @@ func (t *Mimir) initDistributorService() (serv services.Service, err error) { // ruler's dependency) canJoinDistributorsRing := t.Cfg.isAnyModuleEnabled(Distributor, Write, All) - t.Cfg.Distributor.PreferStreamingChunks = t.Cfg.Querier.PreferStreamingChunks + t.Cfg.Distributor.PreferStreamingChunksFromIngesters = t.Cfg.Querier.PreferStreamingChunksFromIngesters t.Cfg.Distributor.StreamingChunksPerIngesterSeriesBufferSize = t.Cfg.Querier.StreamingChunksPerIngesterSeriesBufferSize t.Cfg.Distributor.MinimizeIngesterRequests = t.Cfg.Querier.MinimizeIngesterRequests t.Cfg.Distributor.MinimiseIngesterRequestsHedgingDelay = t.Cfg.Querier.MinimiseIngesterRequestsHedgingDelay diff --git a/pkg/querier/block.go b/pkg/querier/block.go index f0a267e73ef..d7259f9793e 100644 --- a/pkg/querier/block.go +++ b/pkg/querier/block.go @@ -46,8 +46,7 @@ func convertMatchersToLabelMatcher(matchers []*labels.Matcher) []storepb.LabelMa // Implementation of storage.SeriesSet, based on individual responses from store client. type blockQuerierSeriesSet struct { - series []*storepb.Series - warnings storage.Warnings + series []*storepb.Series // next response to process next int @@ -88,7 +87,7 @@ func (bqss *blockQuerierSeriesSet) Err() error { } func (bqss *blockQuerierSeriesSet) Warnings() storage.Warnings { - return bqss.warnings + return nil } // newBlockQuerierSeries makes a new blockQuerierSeries. Input labels must be already sorted by name. @@ -109,15 +108,37 @@ func (bqs *blockQuerierSeries) Labels() labels.Labels { return bqs.labels } -func (bqs *blockQuerierSeries) Iterator(_ chunkenc.Iterator) chunkenc.Iterator { +func (bqs *blockQuerierSeries) Iterator(reuse chunkenc.Iterator) chunkenc.Iterator { if len(bqs.chunks) == 0 { // should not happen in practice, but we have a unit test for it return series.NewErrIterator(errors.New("no chunks")) } - its := make([]iteratorWithMaxTime, 0, len(bqs.chunks)) + it, err := newBlockQuerierSeriesIterator(reuse, bqs.Labels(), bqs.chunks) + if err != nil { + return series.NewErrIterator(err) + } + + return it +} + +func newBlockQuerierSeriesIterator(reuse chunkenc.Iterator, lbls labels.Labels, chunks []storepb.AggrChunk) (*blockQuerierSeriesIterator, error) { + var it *blockQuerierSeriesIterator + r, ok := reuse.(*blockQuerierSeriesIterator) + if ok { + it = r + it.i = 0 + } else { + it = &blockQuerierSeriesIterator{} + } + if cap(it.iterators) < len(chunks) { + it.iterators = make([]iteratorWithMaxTime, len(chunks)) + } + it.iterators = it.iterators[:len(chunks)] + it.labels = lbls + it.lastT = math.MinInt64 - for _, c := range bqs.chunks { + for i, c := range chunks { var ( ch chunkenc.Chunk err error @@ -130,22 +151,18 @@ func (bqs *blockQuerierSeries) Iterator(_ chunkenc.Iterator) chunkenc.Iterator { case storepb.Chunk_FloatHistogram: ch, err = chunkenc.FromData(chunkenc.EncFloatHistogram, c.Raw.Data) default: - return series.NewErrIterator(errors.Wrapf(err, "failed to initialize chunk from unknown type (%v) encoded raw data (series: %v min time: %d max time: %d)", c.Raw.Type, bqs.Labels(), c.MinTime, c.MaxTime)) + return nil, errors.Wrapf(err, "failed to initialize chunk from unknown type (%v) encoded raw data (series: %v min time: %d max time: %d)", c.Raw.Type, lbls, c.MinTime, c.MaxTime) } if err != nil { - return series.NewErrIterator(errors.Wrapf(err, "failed to initialize chunk from %v type encoded raw data (series: %v min time: %d max time: %d)", c.Raw.Type, bqs.Labels(), c.MinTime, c.MaxTime)) + return nil, errors.Wrapf(err, "failed to initialize chunk from %v type encoded raw data (series: %v min time: %d max time: %d)", c.Raw.Type, lbls, c.MinTime, c.MaxTime) } - it := ch.Iterator(nil) - its = append(its, iteratorWithMaxTime{it, c.MaxTime}) + it.iterators[i].Iterator = ch.Iterator(it.iterators[i].Iterator) + it.iterators[i].maxT = c.MaxTime } - return newBlockQuerierSeriesIterator(bqs.Labels(), its) -} - -func newBlockQuerierSeriesIterator(labels labels.Labels, its []iteratorWithMaxTime) *blockQuerierSeriesIterator { - return &blockQuerierSeriesIterator{labels: labels, iterators: its, lastT: math.MinInt64} + return it, nil } // iteratorWithMaxTime is an iterator which is aware of the maxT of its embedded iterator. diff --git a/pkg/querier/block_streaming.go b/pkg/querier/block_streaming.go new file mode 100644 index 00000000000..803cda8f1fa --- /dev/null +++ b/pkg/querier/block_streaming.go @@ -0,0 +1,292 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package querier + +import ( + "fmt" + "io" + "sort" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + + "github.com/grafana/mimir/pkg/mimirpb" + "github.com/grafana/mimir/pkg/querier/stats" + "github.com/grafana/mimir/pkg/storage/series" + "github.com/grafana/mimir/pkg/storegateway/storegatewaypb" + "github.com/grafana/mimir/pkg/storegateway/storepb" + "github.com/grafana/mimir/pkg/util/limiter" + "github.com/grafana/mimir/pkg/util/spanlogger" + "github.com/grafana/mimir/pkg/util/validation" +) + +// Implementation of storage.SeriesSet, based on individual responses from store client. +type blockStreamingQuerierSeriesSet struct { + series []*storepb.StreamingSeries + streamReader chunkStreamReader + + // next response to process + nextSeriesIndex int + + currSeries storage.Series +} + +type chunkStreamReader interface { + GetChunks(seriesIndex uint64) ([]storepb.AggrChunk, error) +} + +func (bqss *blockStreamingQuerierSeriesSet) Next() bool { + bqss.currSeries = nil + + if bqss.nextSeriesIndex >= len(bqss.series) { + return false + } + + currLabels := mimirpb.FromLabelAdaptersToLabels(bqss.series[bqss.nextSeriesIndex].Labels) + seriesIdxStart := bqss.nextSeriesIndex // First series in this group. We might merge with more below. + bqss.nextSeriesIndex++ + + // Chunks may come in multiple responses, but as soon as the response has chunks for a new series, + // we can stop searching. Series are sorted. See documentation for StoreClient.Series call for details. + // The actually merging of chunks happens in the Iterator() call where chunks are fetched. + for bqss.nextSeriesIndex < len(bqss.series) && labels.Compare(currLabels, mimirpb.FromLabelAdaptersToLabels(bqss.series[bqss.nextSeriesIndex].Labels)) == 0 { + bqss.nextSeriesIndex++ + } + + bqss.currSeries = newBlockStreamingQuerierSeries(currLabels, seriesIdxStart, bqss.nextSeriesIndex-1, bqss.streamReader) + return true +} + +func (bqss *blockStreamingQuerierSeriesSet) At() storage.Series { + return bqss.currSeries +} + +func (bqss *blockStreamingQuerierSeriesSet) Err() error { + return nil +} + +func (bqss *blockStreamingQuerierSeriesSet) Warnings() storage.Warnings { + return nil +} + +// newBlockStreamingQuerierSeries makes a new blockQuerierSeries. Input labels must be already sorted by name. +func newBlockStreamingQuerierSeries(lbls labels.Labels, seriesIdxStart, seriesIdxEnd int, streamReader chunkStreamReader) *blockStreamingQuerierSeries { + return &blockStreamingQuerierSeries{ + labels: lbls, + seriesIdxStart: seriesIdxStart, + seriesIdxEnd: seriesIdxEnd, + streamReader: streamReader, + } +} + +type blockStreamingQuerierSeries struct { + labels labels.Labels + seriesIdxStart, seriesIdxEnd int + streamReader chunkStreamReader +} + +func (bqs *blockStreamingQuerierSeries) Labels() labels.Labels { + return bqs.labels +} + +func (bqs *blockStreamingQuerierSeries) Iterator(reuse chunkenc.Iterator) chunkenc.Iterator { + // Fetch the chunks from the stream. + var allChunks []storepb.AggrChunk + for i := bqs.seriesIdxStart; i <= bqs.seriesIdxEnd; i++ { + chks, err := bqs.streamReader.GetChunks(uint64(i)) + if err != nil { + return series.NewErrIterator(err) + } + allChunks = append(allChunks, chks...) + } + if len(allChunks) == 0 { + // should not happen in practice, but we have a unit test for it + return series.NewErrIterator(errors.New("no chunks")) + } + + sort.Slice(allChunks, func(i, j int) bool { + return allChunks[i].MinTime < allChunks[j].MinTime + }) + + it, err := newBlockQuerierSeriesIterator(reuse, bqs.Labels(), allChunks) + if err != nil { + return series.NewErrIterator(err) + } + + return it +} + +// storeGatewayStreamReader is responsible for managing the streaming of chunks from a storegateway and buffering +// chunks in memory until they are consumed by the PromQL engine. +type storeGatewayStreamReader struct { + client storegatewaypb.StoreGateway_SeriesClient + expectedSeriesCount int + queryLimiter *limiter.QueryLimiter + stats *stats.Stats + log log.Logger + + seriesChunksChan chan *storepb.StreamingChunksBatch + chunksBatch []*storepb.StreamingChunks + errorChan chan error +} + +func newStoreGatewayStreamReader(client storegatewaypb.StoreGateway_SeriesClient, expectedSeriesCount int, queryLimiter *limiter.QueryLimiter, stats *stats.Stats, log log.Logger) *storeGatewayStreamReader { + return &storeGatewayStreamReader{ + client: client, + expectedSeriesCount: expectedSeriesCount, + queryLimiter: queryLimiter, + stats: stats, + log: log, + } +} + +// Close cleans up all resources associated with this storeGatewayStreamReader. +// This method should only be called if StartBuffering is not called. +func (s *storeGatewayStreamReader) Close() { + if err := s.client.CloseSend(); err != nil { + level.Warn(s.log).Log("msg", "closing storegateway client stream failed", "err", err) + } +} + +// StartBuffering begins streaming series' chunks from the storegateway associated with +// this storeGatewayStreamReader. Once all series have been consumed with GetChunks, all resources +// associated with this storeGatewayStreamReader are cleaned up. +// If an error occurs while streaming, a subsequent call to GetChunks will return an error. +// To cancel buffering, cancel the context associated with this storeGatewayStreamReader's storegatewaypb.StoreGateway_SeriesClient. +func (s *storeGatewayStreamReader) StartBuffering() { + // Important: to ensure that the goroutine does not become blocked and leak, the goroutine must only ever write to errorChan at most once. + s.errorChan = make(chan error, 1) + s.seriesChunksChan = make(chan *storepb.StreamingChunksBatch, 1) + + ctxDone := s.client.Context().Done() + go func() { + log, _ := spanlogger.NewWithLogger(s.client.Context(), s.log, "storeGatewayStreamReader.StartBuffering") + + defer func() { + s.Close() + close(s.seriesChunksChan) + close(s.errorChan) + log.Finish() + }() + + onError := func(err error) { + s.errorChan <- err + log.Error(err) + } + + totalSeries := 0 + totalChunks := 0 + + for { + msg, err := s.client.Recv() + if err != nil { + if errors.Is(err, io.EOF) { + if totalSeries < s.expectedSeriesCount { + onError(fmt.Errorf("expected to receive %v series, but got EOF after receiving %v series", s.expectedSeriesCount, totalSeries)) + } else { + level.Debug(log).Log("msg", "finished streaming", "series", totalSeries, "chunks", totalChunks) + } + } else { + onError(err) + } + + return + } + + c := msg.GetStreamingChunks() + if c == nil { + onError(fmt.Errorf("expected to receive StreamingSeriesChunks, but got something else")) + return + } + + if len(c.Series) == 0 { + continue + } + + totalSeries += len(c.Series) + if totalSeries > s.expectedSeriesCount { + onError(fmt.Errorf("expected to receive only %v series, but received at least %v series", s.expectedSeriesCount, totalSeries)) + return + } + + chunkBytes := 0 + numChunks := 0 + for _, s := range c.Series { + numChunks += len(s.Chunks) + for _, ch := range s.Chunks { + chunkBytes += ch.Size() + } + } + totalChunks += numChunks + if err := s.queryLimiter.AddChunks(numChunks); err != nil { + onError(validation.LimitError(err.Error())) + return + } + if err := s.queryLimiter.AddChunkBytes(chunkBytes); err != nil { + onError(validation.LimitError(err.Error())) + return + } + + s.stats.AddFetchedChunks(uint64(numChunks)) + s.stats.AddFetchedChunkBytes(uint64(chunkBytes)) + + select { + case <-ctxDone: + // Why do we abort if the context is done? + // We want to make sure that this goroutine is never leaked. + // This goroutine could be leaked if nothing is reading from the buffer, but this method is still trying to send + // more series to a full buffer: it would block forever. + // So, here, we try to send the series to the buffer if we can, but if the context is cancelled, then we give up. + // This only works correctly if the context is cancelled when the query request is complete or cancelled, + // which is true at the time of writing. + onError(s.client.Context().Err()) + return + case s.seriesChunksChan <- c: + // Batch enqueued successfully, nothing else to do for this batch. + } + } + }() +} + +// GetChunks returns the chunks for the series with index seriesIndex. +// This method must be called with monotonically increasing values of seriesIndex. +func (s *storeGatewayStreamReader) GetChunks(seriesIndex uint64) ([]storepb.AggrChunk, error) { + if len(s.chunksBatch) == 0 { + chks, channelOpen := <-s.seriesChunksChan + + if !channelOpen { + // If there's an error, report it. + select { + case err, haveError := <-s.errorChan: + if haveError { + if _, ok := err.(validation.LimitError); ok { + return nil, err + } + return nil, errors.Wrapf(err, "attempted to read series at index %v from stream, but the stream has failed", seriesIndex) + } + default: + } + + return nil, fmt.Errorf("attempted to read series at index %v from stream, but the stream has already been exhausted", seriesIndex) + } + + s.chunksBatch = chks.Series + } + + chks := s.chunksBatch[0] + if len(s.chunksBatch) > 1 { + s.chunksBatch = s.chunksBatch[1:] + } else { + s.chunksBatch = nil + } + + if chks.SeriesIndex != seriesIndex { + return nil, fmt.Errorf("attempted to read series at index %v from stream, but the stream has series with index %v", seriesIndex, chks.SeriesIndex) + } + + return chks.Chunks, nil +} diff --git a/pkg/querier/block_streaming_test.go b/pkg/querier/block_streaming_test.go new file mode 100644 index 00000000000..1a87f8a220b --- /dev/null +++ b/pkg/querier/block_streaming_test.go @@ -0,0 +1,547 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package querier + +import ( + "context" + "errors" + "fmt" + "io" + "math" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" + "google.golang.org/grpc/metadata" + + "github.com/grafana/mimir/pkg/mimirpb" + "github.com/grafana/mimir/pkg/querier/stats" + "github.com/grafana/mimir/pkg/storegateway/storepb" + "github.com/grafana/mimir/pkg/util/limiter" + "github.com/grafana/mimir/pkg/util/test" +) + +func TestBlockStreamingQuerierSeriesSet(t *testing.T) { + cases := map[string]struct { + input []testSeries + expResult []testSeries + errorChunkStreamer bool + }{ + "simple case of one series": { + input: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar"), + values: []testSample{{1, 1}}, + }, + }, + expResult: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar"), + values: []testSample{{1, 1}}, + }, + }, + }, + "multiple unique series": { + input: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}}, + }, + { + lbls: labels.FromStrings("foo", "bar2"), + values: []testSample{{2, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + }, + expResult: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}}, + }, + { + lbls: labels.FromStrings("foo", "bar2"), + values: []testSample{{2, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + }, + }, + "multiple entries of the same series": { + input: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}}, + }, + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{6, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + }, + expResult: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}, {6, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + }, + }, + "multiple entries of the same series again": { + input: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}}, + }, + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{6, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{4, 3}, {5, 3}, {6, 3}}, + }, + }, + expResult: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}, {6, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}, {4, 3}, {5, 3}, {6, 3}}, + }, + }, + }, + "multiple unique series but with erroring chunk streamer": { + errorChunkStreamer: true, + input: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + values: []testSample{{1, 1}, {2, 1}, {5, 10}}, + }, + { + lbls: labels.FromStrings("foo", "bar2"), + values: []testSample{{2, 2}, {9, 2}}, + }, + { + lbls: labels.FromStrings("foo", "bar3"), + values: []testSample{{3, 3}}, + }, + }, + expResult: []testSeries{ + { + lbls: labels.FromStrings("foo", "bar1"), + }, + { + lbls: labels.FromStrings("foo", "bar2"), + }, + { + lbls: labels.FromStrings("foo", "bar3"), + }, + }, + }, + } + + for name, c := range cases { + t.Run(name, func(t *testing.T) { + ss := &blockStreamingQuerierSeriesSet{streamReader: &mockChunkStreamer{series: c.input, causeError: c.errorChunkStreamer}} + for _, s := range c.input { + ss.series = append(ss.series, &storepb.StreamingSeries{ + Labels: mimirpb.FromLabelsToLabelAdapters(s.lbls), + }) + } + idx := 0 + var it chunkenc.Iterator + for ss.Next() { + s := ss.At() + require.Equal(t, c.expResult[idx].lbls, s.Labels()) + it = s.Iterator(it) + if c.errorChunkStreamer { + require.EqualError(t, it.Err(), "mocked error") + } else { + var actSamples []testSample + for it.Next() != chunkenc.ValNone { + ts, val := it.At() + actSamples = append(actSamples, testSample{t: ts, v: val}) + } + require.Equal(t, c.expResult[idx].values, actSamples) + require.NoError(t, it.Err()) + } + idx++ + } + require.NoError(t, ss.Err()) + require.Equal(t, len(c.expResult), idx) + }) + } +} + +type testSeries struct { + lbls labels.Labels + values []testSample +} + +type testSample struct { + t int64 + v float64 +} + +type mockChunkStreamer struct { + series []testSeries + next int + causeError bool +} + +func (m *mockChunkStreamer) GetChunks(seriesIndex uint64) ([]storepb.AggrChunk, error) { + if m.causeError { + return nil, fmt.Errorf("mocked error") + } + if m.next >= len(m.series) { + return nil, fmt.Errorf("out of chunks") + } + + if uint64(m.next) != seriesIndex { + return nil, fmt.Errorf("asked for the wrong series, exp: %d, got %d", m.next, seriesIndex) + } + + chk := chunkenc.NewXORChunk() + app, err := chk.Appender() + if err != nil { + return nil, err + } + + samples := m.series[m.next].values + mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) + for _, s := range samples { + app.Append(s.t, s.v) + if s.t < mint { + mint = s.t + } + if s.t > maxt { + maxt = s.t + } + } + + m.next++ + + return []storepb.AggrChunk{{ + MinTime: mint, + MaxTime: maxt, + Raw: &storepb.Chunk{Data: chk.Bytes()}, + }}, nil +} + +func TestStoreGatewayStreamReader_HappyPaths(t *testing.T) { + series0 := []storepb.AggrChunk{createChunk(t, 1000, 1)} + series1 := []storepb.AggrChunk{createChunk(t, 1000, 2)} + series2 := []storepb.AggrChunk{createChunk(t, 1000, 3)} + series3 := []storepb.AggrChunk{createChunk(t, 1000, 4)} + series4 := []storepb.AggrChunk{createChunk(t, 1000, 5)} + + testCases := map[string]struct { + batches []storepb.StreamingChunksBatch + }{ + "single series per batch": { + batches: []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: series0}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 1, Chunks: series1}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 2, Chunks: series2}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 3, Chunks: series3}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 4, Chunks: series4}}}, + }, + }, + "multiple series per batch": { + batches: []storepb.StreamingChunksBatch{ + { + Series: []*storepb.StreamingChunks{ + {SeriesIndex: 0, Chunks: series0}, + {SeriesIndex: 1, Chunks: series1}, + {SeriesIndex: 2, Chunks: series2}, + }, + }, + { + Series: []*storepb.StreamingChunks{ + {SeriesIndex: 3, Chunks: series3}, + {SeriesIndex: 4, Chunks: series4}, + }, + }, + }, + }, + "empty batches": { + batches: []storepb.StreamingChunksBatch{ + { + Series: []*storepb.StreamingChunks{ + {SeriesIndex: 0, Chunks: series0}, + {SeriesIndex: 1, Chunks: series1}, + {SeriesIndex: 2, Chunks: series2}, + }, + }, + {}, + { + Series: []*storepb.StreamingChunks{ + {SeriesIndex: 3, Chunks: series3}, + {SeriesIndex: 4, Chunks: series4}, + }, + }, + {}, + }, + }, + } + + for name, testCase := range testCases { + t.Run(name, func(t *testing.T) { + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: testCase.batches} + reader := newStoreGatewayStreamReader(mockClient, 5, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + for i, expected := range [][]storepb.AggrChunk{series0, series1, series2, series3, series4} { + actual, err := reader.GetChunks(uint64(i)) + require.NoError(t, err) + require.Equalf(t, expected, actual, "received unexpected chunk for series index %v", i) + } + + require.Eventually(t, func() bool { + return mockClient.closed.Load() + }, time.Second, 10*time.Millisecond) + }) + } +} + +func TestStoreGatewayStreamReader_AbortsWhenContextCancelled(t *testing.T) { + // Ensure that the buffering goroutine is not leaked after context cancellation. + test.VerifyNoLeak(t) + + // Create multiple batches to ensure that the buffering goroutine becomes blocked waiting to send further chunks to GetChunks(). + batches := []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 1.23)}}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 1, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 4.56)}}}}, + {Series: []*storepb.StreamingChunks{{SeriesIndex: 2, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 7.89)}}}}, + } + + ctx, cancel := context.WithCancel(context.Background()) + mockClient := &mockStoreGatewayQueryStreamClient{ctx: ctx, batches: batches} + + reader := newStoreGatewayStreamReader(mockClient, 3, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + cancel() + reader.StartBuffering() + + for i := 0; i < 3; i++ { + _, err := reader.GetChunks(uint64(i)) + + if errors.Is(err, context.Canceled) { + break + } + + require.NoError(t, err) + + if i == 2 { + require.Fail(t, "expected GetChunks to report context cancellation error before reaching end of stream") + } + } + + require.True(t, mockClient.closed.Load(), "expected gRPC client to be closed after context cancelled") +} + +func TestStoreGatewayStreamReader_ReadingSeriesOutOfOrder(t *testing.T) { + batches := []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 1.23)}}}}, + } + + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: batches} + reader := newStoreGatewayStreamReader(mockClient, 1, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + s, err := reader.GetChunks(1) + require.Nil(t, s) + require.EqualError(t, err, "attempted to read series at index 1 from stream, but the stream has series with index 0") +} + +func TestStoreGatewayStreamReader_ReadingMoreSeriesThanAvailable(t *testing.T) { + firstSeries := []storepb.AggrChunk{createChunk(t, 1000, 1.23)} + batches := []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: firstSeries}}}, + } + + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: batches} + reader := newStoreGatewayStreamReader(mockClient, 1, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + s, err := reader.GetChunks(0) + require.NoError(t, err) + require.Equal(t, s, firstSeries) + + s, err = reader.GetChunks(1) + require.Nil(t, s) + require.EqualError(t, err, "attempted to read series at index 1 from stream, but the stream has already been exhausted") +} + +func TestStoreGatewayStreamReader_ReceivedFewerSeriesThanExpected(t *testing.T) { + firstSeries := []storepb.AggrChunk{createChunk(t, 1000, 1.23)} + batches := []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: firstSeries}}}, + } + + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: batches} + reader := newStoreGatewayStreamReader(mockClient, 3, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + s, err := reader.GetChunks(0) + require.NoError(t, err) + require.Equal(t, s, firstSeries) + + s, err = reader.GetChunks(1) + require.Nil(t, s) + require.EqualError(t, err, "attempted to read series at index 1 from stream, but the stream has failed: expected to receive 3 series, but got EOF after receiving 1 series") + + require.True(t, mockClient.closed.Load(), "expected gRPC client to be closed after failure") +} + +func TestStoreGatewayStreamReader_ReceivedMoreSeriesThanExpected(t *testing.T) { + batches := []storepb.StreamingChunksBatch{ + { + Series: []*storepb.StreamingChunks{ + {SeriesIndex: 0, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 1.23)}}, + {SeriesIndex: 1, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 1.23)}}, + {SeriesIndex: 2, Chunks: []storepb.AggrChunk{createChunk(t, 1000, 1.23)}}, + }, + }, + } + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: batches} + reader := newStoreGatewayStreamReader(mockClient, 1, limiter.NewQueryLimiter(0, 0, 0, nil), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + s, err := reader.GetChunks(0) + require.Nil(t, s) + require.EqualError(t, err, "attempted to read series at index 0 from stream, but the stream has failed: expected to receive only 1 series, but received at least 3 series") + + require.True(t, mockClient.closed.Load(), "expected gRPC client to be closed after receiving more series than expected") +} + +func TestStoreGatewayStreamReader_ChunksLimits(t *testing.T) { + testCases := map[string]struct { + maxChunks int + maxChunkBytes int + expectedError string + }{ + "query under both limits": { + maxChunks: 4, + maxChunkBytes: 200, + expectedError: "", + }, + "query selects too many chunks": { + maxChunks: 2, + maxChunkBytes: 200, + expectedError: "the query exceeded the maximum number of chunks (limit: 2 chunks) (err-mimir-max-chunks-per-query). Consider reducing the time range and/or number of series selected by the query. One way to reduce the number of selected series is to add more label matchers to the query. Otherwise, to adjust the related per-tenant limit, configure -querier.max-fetched-chunks-per-query, or contact your service administrator.", + }, + "query selects too many chunk bytes": { + maxChunks: 4, + maxChunkBytes: 50, + expectedError: "the query exceeded the aggregated chunks size limit (limit: 50 bytes) (err-mimir-max-chunks-bytes-per-query). Consider reducing the time range and/or number of series selected by the query. One way to reduce the number of selected series is to add more label matchers to the query. Otherwise, to adjust the related per-tenant limit, configure -querier.max-fetched-chunk-bytes-per-query, or contact your service administrator.", + }, + } + + for name, testCase := range testCases { + t.Run(name, func(t *testing.T) { + batches := []storepb.StreamingChunksBatch{ + {Series: []*storepb.StreamingChunks{{SeriesIndex: 0, Chunks: []storepb.AggrChunk{ + createChunk(t, 1000, 1.23), + createChunk(t, 1100, 1.23), + createChunk(t, 1200, 1.23), + }}}}, + } + + mockClient := &mockStoreGatewayQueryStreamClient{ctx: context.Background(), batches: batches} + queryMetrics := stats.NewQueryMetrics(prometheus.NewPedanticRegistry()) + reader := newStoreGatewayStreamReader(mockClient, 1, limiter.NewQueryLimiter(0, testCase.maxChunkBytes, testCase.maxChunks, queryMetrics), &stats.Stats{}, log.NewNopLogger()) + reader.StartBuffering() + + _, err := reader.GetChunks(0) + + if testCase.expectedError == "" { + require.NoError(t, err) + } else { + require.EqualError(t, err, testCase.expectedError) + } + + require.Eventually(t, mockClient.closed.Load, time.Second, 10*time.Millisecond, "expected gRPC client to be closed") + }) + } +} + +func createChunk(t *testing.T, time int64, value float64) storepb.AggrChunk { + promChunk := chunkenc.NewXORChunk() + app, err := promChunk.Appender() + require.NoError(t, err) + + app.Append(time, value) + + return storepb.AggrChunk{ + MinTime: time, + MaxTime: time, + Raw: &storepb.Chunk{ + Type: storepb.Chunk_XOR, + Data: promChunk.Bytes(), + }, + } +} + +type mockStoreGatewayQueryStreamClient struct { + ctx context.Context + batches []storepb.StreamingChunksBatch + closed atomic.Bool +} + +func (m *mockStoreGatewayQueryStreamClient) Recv() (*storepb.SeriesResponse, error) { + if len(m.batches) == 0 { + return nil, io.EOF + } + + batch := m.batches[0] + m.batches = m.batches[1:] + + return storepb.NewStreamingChunksResponse(&batch), nil +} + +func (m *mockStoreGatewayQueryStreamClient) Header() (metadata.MD, error) { + panic("not supported on mock") +} + +func (m *mockStoreGatewayQueryStreamClient) Trailer() metadata.MD { + panic("not supported on mock") +} + +func (m *mockStoreGatewayQueryStreamClient) CloseSend() error { + m.closed.Store(true) + return nil +} + +func (m *mockStoreGatewayQueryStreamClient) Context() context.Context { + return m.ctx +} + +func (m *mockStoreGatewayQueryStreamClient) SendMsg(interface{}) error { + panic("not supported on mock") +} + +func (m *mockStoreGatewayQueryStreamClient) RecvMsg(interface{}) error { + panic("not supported on mock") +} diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index d8725bc93d5..0a5b08aa883 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -51,6 +51,7 @@ import ( util_log "github.com/grafana/mimir/pkg/util/log" "github.com/grafana/mimir/pkg/util/math" "github.com/grafana/mimir/pkg/util/spanlogger" + "github.com/grafana/mimir/pkg/util/validation" ) const ( @@ -142,13 +143,14 @@ func newBlocksStoreQueryableMetrics(reg prometheus.Registerer) *blocksStoreQuery type BlocksStoreQueryable struct { services.Service - stores BlocksStoreSet - finder BlocksFinder - consistency *BlocksConsistencyChecker - logger log.Logger - queryStoreAfter time.Duration - metrics *blocksStoreQueryableMetrics - limits BlocksStoreLimits + stores BlocksStoreSet + finder BlocksFinder + consistency *BlocksConsistencyChecker + logger log.Logger + queryStoreAfter time.Duration + metrics *blocksStoreQueryableMetrics + limits BlocksStoreLimits + streamingChunksBatchSize uint64 // Subservices manager. subservices *services.Manager @@ -161,6 +163,7 @@ func NewBlocksStoreQueryable( consistency *BlocksConsistencyChecker, limits BlocksStoreLimits, queryStoreAfter time.Duration, + streamingChunksBatchSize uint64, logger log.Logger, reg prometheus.Registerer, ) (*BlocksStoreQueryable, error) { @@ -170,15 +173,16 @@ func NewBlocksStoreQueryable( } q := &BlocksStoreQueryable{ - stores: stores, - finder: finder, - consistency: consistency, - queryStoreAfter: queryStoreAfter, - logger: logger, - subservices: manager, - subservicesWatcher: services.NewFailureWatcher(), - metrics: newBlocksStoreQueryableMetrics(reg), - limits: limits, + stores: stores, + finder: finder, + consistency: consistency, + queryStoreAfter: queryStoreAfter, + logger: logger, + subservices: manager, + subservicesWatcher: services.NewFailureWatcher(), + metrics: newBlocksStoreQueryableMetrics(reg), + limits: limits, + streamingChunksBatchSize: streamingChunksBatchSize, } q.Service = services.NewBasicService(q.starting, q.running, q.stopping) @@ -260,7 +264,12 @@ func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegatewa reg, ) - return NewBlocksStoreQueryable(stores, finder, consistency, limits, querierCfg.QueryStoreAfter, logger, reg) + streamingBufferSize := querierCfg.StreamingChunksPerStoreGatewaySeriesBufferSize + if !querierCfg.PreferStreamingChunksFromStoreGateways { + streamingBufferSize = 0 + } + + return NewBlocksStoreQueryable(stores, finder, consistency, limits, querierCfg.QueryStoreAfter, streamingBufferSize, logger, reg) } func (q *BlocksStoreQueryable) starting(ctx context.Context) error { @@ -300,30 +309,32 @@ func (q *BlocksStoreQueryable) Querier(ctx context.Context, mint, maxt int64) (s } return &blocksStoreQuerier{ - ctx: ctx, - minT: mint, - maxT: maxt, - userID: userID, - finder: q.finder, - stores: q.stores, - metrics: q.metrics, - limits: q.limits, - consistency: q.consistency, - logger: q.logger, - queryStoreAfter: q.queryStoreAfter, + ctx: ctx, + minT: mint, + maxT: maxt, + userID: userID, + finder: q.finder, + stores: q.stores, + metrics: q.metrics, + limits: q.limits, + streamingChunksBatchSize: q.streamingChunksBatchSize, + consistency: q.consistency, + logger: q.logger, + queryStoreAfter: q.queryStoreAfter, }, nil } type blocksStoreQuerier struct { - ctx context.Context - minT, maxT int64 - userID string - finder BlocksFinder - stores BlocksStoreSet - metrics *blocksStoreQueryableMetrics - consistency *BlocksConsistencyChecker - limits BlocksStoreLimits - logger log.Logger + ctx context.Context + minT, maxT int64 + userID string + finder BlocksFinder + stores BlocksStoreSet + metrics *blocksStoreQueryableMetrics + consistency *BlocksConsistencyChecker + limits BlocksStoreLimits + streamingChunksBatchSize uint64 + logger log.Logger // If set, the querier manipulates the max time to not be greater than // "now - queryStoreAfter" so that most recent blocks are not queried. @@ -433,6 +444,7 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...* convertedMatchers = convertMatchersToLabelMatcher(matchers) resSeriesSets = []storage.SeriesSet(nil) resWarnings = storage.Warnings(nil) + streamStarters []func() ) shard, _, err := sharding.ShardFromMatchers(matchers) @@ -441,13 +453,14 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...* } queryFunc := func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error) { - seriesSets, queriedBlocks, warnings, err := q.fetchSeriesFromStores(spanCtx, sp, clients, minT, maxT, convertedMatchers) + seriesSets, queriedBlocks, warnings, startStreamingChunks, err := q.fetchSeriesFromStores(spanCtx, sp, clients, minT, maxT, convertedMatchers) if err != nil { return nil, err } resSeriesSets = append(resSeriesSets, seriesSets...) resWarnings = append(resWarnings, warnings...) + streamStarters = append(streamStarters, startStreamingChunks) return queriedBlocks, nil } @@ -457,6 +470,11 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...* return storage.ErrSeriesSet(err) } + // If this was a streaming call, start fetching streaming chunks here. + for _, ss := range streamStarters { + ss() + } + if len(resSeriesSets) == 0 { storage.EmptySeriesSet() } @@ -675,7 +693,11 @@ func canBlockWithCompactorShardIndexContainQueryShard(queryShardIndex, queryShar // Errors while creating storepb.SeriesRequest, context cancellation, and unprocessable // requests to the store-gateways (e.g., if a chunk or series limit is hit) are // considered serious errors. All other errors are not returned, but they give rise to fetch retrials. -func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *storage.SelectHints, clients map[BlocksStoreClient][]ulid.ULID, minT int64, maxT int64, convertedMatchers []storepb.LabelMatcher) ([]storage.SeriesSet, []ulid.ULID, storage.Warnings, error) { +// +// In case of a successful run, fetchSeriesFromStores returns a startStreamingChunks function to start streaming +// chunks for the fetched series iff it was a streaming call for series+chunks. startStreamingChunks must be called +// before iterating on the series. +func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *storage.SelectHints, clients map[BlocksStoreClient][]ulid.ULID, minT int64, maxT int64, convertedMatchers []storepb.LabelMatcher) (_ []storage.SeriesSet, _ []ulid.ULID, _ storage.Warnings, startStreamingChunks func(), _ error) { var ( reqCtx = grpc_metadata.AppendToOutgoingContext(ctx, storegateway.GrpcContextMetadataTenantID, q.userID) g, gCtx = errgroup.WithContext(reqCtx) @@ -686,6 +708,8 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor spanLog = spanlogger.FromContext(ctx, q.logger) queryLimiter = limiter.QueryLimiterFromContextWithFallback(ctx) reqStats = stats.FromContext(ctx) + streamReaders []*storeGatewayStreamReader + streams []storegatewaypb.StoreGateway_SeriesClient ) // Concurrently fetch series from all clients. @@ -701,12 +725,18 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor // But this is an acceptable workaround for now. skipChunks := sp != nil && sp.Func == "series" - req, err := createSeriesRequest(minT, maxT, convertedMatchers, skipChunks, blockIDs) + req, err := createSeriesRequest(minT, maxT, convertedMatchers, skipChunks, blockIDs, q.streamingChunksBatchSize) if err != nil { return errors.Wrapf(err, "failed to create series request") } - stream, err := c.Series(gCtx, req) + stream, err := c.Series(reqCtx, req) + if err == nil { + mtx.Lock() + streams = append(streams, stream) + mtx.Unlock() + err = gCtx.Err() + } if err != nil { if shouldStopQueryFunc(err) { return err @@ -716,7 +746,9 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor return nil } + // A storegateway client will only fill either of mySeries or myStreamingSeries, and not both. mySeries := []*storepb.Series(nil) + myStreamingSeries := []*storepb.StreamingSeries(nil) myWarnings := storage.Warnings(nil) myQueriedBlocks := []ulid.ULID(nil) indexBytesFetched := uint64(0) @@ -741,7 +773,7 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor return nil } - // Response may either contain series, warning or hints. + // Response may either contain series, streaming series, warning or hints. if s := resp.GetSeries(); s != nil { mySeries = append(mySeries, s) @@ -781,28 +813,60 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor if s := resp.GetStats(); s != nil { indexBytesFetched += s.FetchedIndexBytes } - } - numSeries := len(mySeries) - chunksFetched, chunkBytes := countChunksAndBytes(mySeries...) + if ss := resp.GetStreamingSeries(); ss != nil { + for _, s := range ss.Series { + // Add series fingerprint to query limiter; will return error if we are over the limit + limitErr := queryLimiter.AddSeries(s.Labels) + if limitErr != nil { + return validation.LimitError(limitErr.Error()) + } + } + myStreamingSeries = append(myStreamingSeries, ss.Series...) + if ss.IsEndOfSeriesStream { + // We expect "end of stream" to be sent after the hints and the stats have been sent. + break + } + } + } - reqStats.AddFetchedSeries(uint64(numSeries)) - reqStats.AddFetchedChunkBytes(uint64(chunkBytes)) - reqStats.AddFetchedChunks(uint64(chunksFetched)) reqStats.AddFetchedIndexBytes(indexBytesFetched) - - level.Debug(spanLog).Log("msg", "received series from store-gateway", - "instance", c.RemoteAddress(), - "fetched series", numSeries, - "fetched chunk bytes", chunkBytes, - "fetched chunks", chunksFetched, - "fetched index bytes", indexBytesFetched, - "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), - "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) + var streamReader *storeGatewayStreamReader + if len(mySeries) > 0 { + chunksFetched, chunkBytes := countChunksAndBytes(mySeries...) + + reqStats.AddFetchedSeries(uint64(len(mySeries))) + reqStats.AddFetchedChunkBytes(uint64(chunkBytes)) + reqStats.AddFetchedChunks(uint64(chunksFetched)) + + level.Debug(spanLog).Log("msg", "received series from store-gateway", + "instance", c.RemoteAddress(), + "fetched series", len(mySeries), + "fetched chunk bytes", chunkBytes, + "fetched chunks", chunksFetched, + "fetched index bytes", indexBytesFetched, + "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), + "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) + } else if len(myStreamingSeries) > 0 { + // FetchedChunks and FetchedChunkBytes are added by the SeriesChunksStreamReader. + reqStats.AddFetchedSeries(uint64(len(myStreamingSeries))) + streamReader = newStoreGatewayStreamReader(stream, len(myStreamingSeries), queryLimiter, reqStats, q.logger) + level.Debug(spanLog).Log("msg", "received streaming series from store-gateway", + "instance", c.RemoteAddress(), + "fetched series", len(myStreamingSeries), + "fetched index bytes", indexBytesFetched, + "requested blocks", strings.Join(convertULIDsToString(blockIDs), " "), + "queried blocks", strings.Join(convertULIDsToString(myQueriedBlocks), " ")) + } // Store the result. mtx.Lock() - seriesSets = append(seriesSets, &blockQuerierSeriesSet{series: mySeries}) + if len(mySeries) > 0 { + seriesSets = append(seriesSets, &blockQuerierSeriesSet{series: mySeries}) + } else if len(myStreamingSeries) > 0 { + seriesSets = append(seriesSets, &blockStreamingQuerierSeriesSet{series: myStreamingSeries, streamReader: streamReader}) + streamReaders = append(streamReaders, streamReader) + } warnings = append(warnings, myWarnings...) queriedBlocks = append(queriedBlocks, myQueriedBlocks...) mtx.Unlock() @@ -813,10 +877,21 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(ctx context.Context, sp *stor // Wait until all client requests complete. if err := g.Wait(); err != nil { - return nil, nil, nil, err + for _, stream := range streams { + if err := stream.CloseSend(); err != nil { + level.Warn(q.logger).Log("msg", "closing storegateway client stream failed", "err", err) + } + } + return nil, nil, nil, nil, err + } + + startStreamingChunks = func() { + for _, sr := range streamReaders { + sr.StartBuffering() + } } - return seriesSets, queriedBlocks, warnings, nil + return seriesSets, queriedBlocks, warnings, startStreamingChunks, nil } func shouldStopQueryFunc(err error) bool { @@ -998,7 +1073,7 @@ func (q *blocksStoreQuerier) fetchLabelValuesFromStore( return valueSets, warnings, queriedBlocks, nil } -func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, skipChunks bool, blockIDs []ulid.ULID) (*storepb.SeriesRequest, error) { +func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, skipChunks bool, blockIDs []ulid.ULID, streamingBatchSize uint64) (*storepb.SeriesRequest, error) { // Selectively query only specific blocks. hints := &hintspb.SeriesRequestHints{ BlockMatchers: []storepb.LabelMatcher{ @@ -1015,12 +1090,17 @@ func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, skip return nil, errors.Wrapf(err, "failed to marshal series request hints") } + if skipChunks { + // We don't do the streaming call if we are not requesting the chunks. + streamingBatchSize = 0 + } return &storepb.SeriesRequest{ - MinTime: minT, - MaxTime: maxT, - Matchers: matchers, - Hints: anyHints, - SkipChunks: skipChunks, + MinTime: minT, + MaxTime: maxT, + Matchers: matchers, + Hints: anyHints, + SkipChunks: skipChunks, + StreamingChunksBatchSize: streamingBatchSize, }, nil } diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index dee7769a3ba..6da3a83af5f 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -32,6 +32,7 @@ import ( "github.com/weaveworks/common/user" "golang.org/x/exp/slices" "google.golang.org/grpc" + "google.golang.org/grpc/metadata" "github.com/grafana/mimir/pkg/mimirpb" "github.com/grafana/mimir/pkg/querier/stats" @@ -135,6 +136,35 @@ func TestBlocksStoreQuerier_Select(t *testing.T) { }, }, }, + "a single store-gateway instance holds the required blocks (single returned series) - multiple chunks per series for stats": { + finderResult: bucketindex.Blocks{ + {ID: block1}, + {ID: block2}, + }, + storeSetResponses: []interface{}{ + map[BlocksStoreClient][]ulid.ULID{ + &storeGatewayClientMock{remoteAddr: "1.1.1.1", mockedSeriesResponses: []*storepb.SeriesResponse{ + mockSeriesResponseWithChunks(metricNameLabel, + createAggrChunkWithSamples(promql.FPoint{T: minT, F: 1}), + createAggrChunkWithSamples(promql.FPoint{T: minT + 1, F: 2}), + ), + mockHintsResponse(block1, block2), + mockStatsResponse(50), + }}: {block1, block2}, + }, + }, + limits: &blocksStoreLimitsMock{}, + queryLimiter: noOpQueryLimiter, + expectedSeries: []seriesResult{ + { + lbls: metricNameLabel, + values: []valueResult{ + {t: minT, v: 1}, + {t: minT + 1, v: 2}, + }, + }, + }, + }, "a single store-gateway instance holds the required blocks (multiple returned series)": { finderResult: bucketindex.Blocks{ {ID: block1}, @@ -764,81 +794,156 @@ func TestBlocksStoreQuerier_Select(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - ctx := limiter.AddQueryLimiterToContext(context.Background(), testData.queryLimiter) - reg := prometheus.NewPedanticRegistry() - stores := &blocksStoreSetMock{mockedResponses: testData.storeSetResponses} - finder := &blocksFinderMock{} - finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(testData.finderResult, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), testData.finderErr) - - q := &blocksStoreQuerier{ - ctx: ctx, - minT: minT, - maxT: maxT, - userID: "user-1", - finder: finder, - stores: stores, - consistency: NewBlocksConsistencyChecker(0, 0, log.NewNopLogger(), nil), - logger: log.NewNopLogger(), - metrics: newBlocksStoreQueryableMetrics(reg), - limits: testData.limits, - } + for _, streaming := range []bool{true, false} { + t.Run(fmt.Sprintf("streaming=%t", streaming), func(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + + // Count the number of series to check the stats later. + // We also make a copy of the testData.storeSetResponses where relevant so that + // we can run the streaming and non-streaming case in any order. + var storeSetResponses []interface{} + seriesCount, chunksCount := 0, 0 + for _, res := range testData.storeSetResponses { + m, ok := res.(map[BlocksStoreClient][]ulid.ULID) + if !ok { + storeSetResponses = append(storeSetResponses, res) + continue + } + newMap := make(map[BlocksStoreClient][]ulid.ULID, len(m)) + for k, v := range m { + mockClient := k.(*storeGatewayClientMock) + for _, sr := range mockClient.mockedSeriesResponses { + if s := sr.GetSeries(); s != nil { + seriesCount++ + chunksCount += len(s.Chunks) + } + } + + shallowCopy := *mockClient + if streaming { + // Convert the storegateway response to streaming response. + shallowCopy.mockedSeriesResponses = generateStreamingResponses(shallowCopy.mockedSeriesResponses) + } + newMap[&shallowCopy] = v + } + storeSetResponses = append(storeSetResponses, newMap) + } - matchers := []*labels.Matcher{ - labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, metricName), - } - if testData.queryShardID != "" { - matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, sharding.ShardLabel, testData.queryShardID)) - } + stores := &blocksStoreSetMock{mockedResponses: storeSetResponses} + finder := &blocksFinderMock{} + finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(testData.finderResult, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), testData.finderErr) + + ctx := limiter.AddQueryLimiterToContext(context.Background(), testData.queryLimiter) + st, ctx := stats.ContextWithEmptyStats(ctx) + q := &blocksStoreQuerier{ + ctx: ctx, + minT: minT, + maxT: maxT, + userID: "user-1", + finder: finder, + stores: stores, + consistency: NewBlocksConsistencyChecker(0, 0, log.NewNopLogger(), nil), + logger: log.NewNopLogger(), + metrics: newBlocksStoreQueryableMetrics(reg), + limits: testData.limits, + } - sp := &storage.SelectHints{Start: minT, End: maxT} - set := q.Select(true, sp, matchers...) - if testData.expectedErr != nil { - assert.ErrorContains(t, set.Err(), testData.expectedErr.Error()) - assert.IsType(t, set.Err(), testData.expectedErr) - assert.False(t, set.Next()) - assert.Nil(t, set.Warnings()) - return - } + matchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, metricName), + } + if testData.queryShardID != "" { + matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, sharding.ShardLabel, testData.queryShardID)) + } - require.NoError(t, set.Err()) - assert.Len(t, set.Warnings(), 0) - - // Read all returned series and their values. - var actualSeries []seriesResult - var it chunkenc.Iterator - for set.Next() { - var actualValues []valueResult - - it = set.At().Iterator(it) - for valType := it.Next(); valType != chunkenc.ValNone; valType = it.Next() { - assert.Equal(t, valType, chunkenc.ValFloat) - t, v := it.At() - actualValues = append(actualValues, valueResult{ - t: t, - v: v, - }) - } + sp := &storage.SelectHints{Start: minT, End: maxT} + set := q.Select(true, sp, matchers...) + if testData.expectedErr != nil { + if streaming && set.Err() == nil { + // In case of streaming, the error can happen during iteration. + var err error + for set.Next() { + it := set.At().Iterator(nil) + for it.Next() != chunkenc.ValNone { // nolint + } + err = it.Err() + if err != nil { + break + } + } + assert.ErrorIs(t, err, testData.expectedErr) + } else { + assert.ErrorContains(t, set.Err(), testData.expectedErr.Error()) + assert.IsType(t, set.Err(), testData.expectedErr) + assert.False(t, set.Next()) + assert.Nil(t, set.Warnings()) + } + return + } - require.NoError(t, it.Err()) + require.NoError(t, set.Err()) + assert.Len(t, set.Warnings(), 0) + + // Read all returned series and their values. + var actualSeries []seriesResult + var it chunkenc.Iterator + for set.Next() { + var actualValues []valueResult + + it = set.At().Iterator(it) + for valType := it.Next(); valType != chunkenc.ValNone; valType = it.Next() { + assert.Equal(t, valType, chunkenc.ValFloat) + t, v := it.At() + actualValues = append(actualValues, valueResult{ + t: t, + v: v, + }) + } + + require.NoError(t, it.Err()) + + actualSeries = append(actualSeries, seriesResult{ + lbls: set.At().Labels(), + values: actualValues, + }) + } + require.NoError(t, set.Err()) + assert.Equal(t, testData.expectedSeries, actualSeries) + assert.Equal(t, seriesCount, int(st.FetchedSeriesCount)) + assert.Equal(t, chunksCount, int(st.FetchedChunksCount)) - actualSeries = append(actualSeries, seriesResult{ - lbls: set.At().Labels(), - values: actualValues, + // Assert on metrics (optional, only for test cases defining it). + if testData.expectedMetrics != "" { + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(testData.expectedMetrics), + "cortex_querier_storegateway_instances_hit_per_query", "cortex_querier_storegateway_refetches_per_query", + "cortex_querier_blocks_found_total", "cortex_querier_blocks_queried_total", "cortex_querier_blocks_with_compactor_shard_but_incompatible_query_shard_total")) + } }) } - require.NoError(t, set.Err()) - assert.Equal(t, testData.expectedSeries, actualSeries) - - // Assert on metrics (optional, only for test cases defining it). - if testData.expectedMetrics != "" { - assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(testData.expectedMetrics), - "cortex_querier_storegateway_instances_hit_per_query", "cortex_querier_storegateway_refetches_per_query", - "cortex_querier_blocks_found_total", "cortex_querier_blocks_queried_total", "cortex_querier_blocks_with_compactor_shard_but_incompatible_query_shard_total")) - } }) } } +func generateStreamingResponses(seriesResponses []*storepb.SeriesResponse) []*storepb.SeriesResponse { + var series, chunks, others, final []*storepb.SeriesResponse + for i, mr := range seriesResponses { + s := mr.GetSeries() + if s != nil { + series = append(series, mockStreamingSeriesBatchResponse(false, s.Labels)) + chunks = append(chunks, mockStreamingSeriesChunksResponse(uint64(len(series)-1), s.Chunks)) + continue + } + others = seriesResponses[i:] + break + } + + final = append(final, series...) + final = append(final, others...) + // End of stream response goes after the hints and stats. + final = append(final, mockStreamingSeriesBatchResponse(true)) + final = append(final, chunks...) + return final +} + func TestBlocksStoreQuerier_Select_cancelledContext(t *testing.T) { const ( metricName = "test_metric" @@ -1704,57 +1809,65 @@ func TestBlocksStoreQuerier_PromQLExecution(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - block1 := ulid.MustNew(1, nil) - block2 := ulid.MustNew(2, nil) + for _, streaming := range []bool{true, false} { + t.Run(fmt.Sprintf("streaming=%t", streaming), func(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + + // Mock the finder to simulate we need to query two blocks. + finder := &blocksFinderMock{ + Service: services.NewIdleService(nil, nil), + } + finder.On("GetBlocks", mock.Anything, "user-1", mock.Anything, mock.Anything).Return(bucketindex.Blocks{ + {ID: block1}, + {ID: block2}, + }, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), error(nil)) + + // Mock the store-gateway response, to simulate the case each block is queried from a different gateway. + gateway1 := &storeGatewayClientMock{remoteAddr: "1.1.1.1", mockedSeriesResponses: append(testData.storeGateway1Responses, mockHintsResponse(block1))} + gateway2 := &storeGatewayClientMock{remoteAddr: "2.2.2.2", mockedSeriesResponses: append(testData.storeGateway2Responses, mockHintsResponse(block2))} + if streaming { + gateway1.mockedSeriesResponses = generateStreamingResponses(gateway1.mockedSeriesResponses) + gateway2.mockedSeriesResponses = generateStreamingResponses(gateway2.mockedSeriesResponses) + } - // Mock the finder to simulate we need to query two blocks. - finder := &blocksFinderMock{ - Service: services.NewIdleService(nil, nil), - } - finder.On("GetBlocks", mock.Anything, "user-1", mock.Anything, mock.Anything).Return(bucketindex.Blocks{ - {ID: block1}, - {ID: block2}, - }, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), error(nil)) - - // Mock the store-gateway response, to simulate the case each block is queried from a different gateway. - gateway1 := &storeGatewayClientMock{remoteAddr: "1.1.1.1", mockedSeriesResponses: append(testData.storeGateway1Responses, mockHintsResponse(block1))} - gateway2 := &storeGatewayClientMock{remoteAddr: "2.2.2.2", mockedSeriesResponses: append(testData.storeGateway2Responses, mockHintsResponse(block2))} - - stores := &blocksStoreSetMock{ - Service: services.NewIdleService(nil, nil), - mockedResponses: []interface{}{ - map[BlocksStoreClient][]ulid.ULID{ - gateway1: {block1}, - gateway2: {block2}, - }, - }, - } + stores := &blocksStoreSetMock{ + Service: services.NewIdleService(nil, nil), + mockedResponses: []interface{}{ + map[BlocksStoreClient][]ulid.ULID{ + gateway1: {block1}, + gateway2: {block2}, + }, + }, + } - // Instantiate the querier that will be executed to run the query. - logger := log.NewNopLogger() - queryable, err := NewBlocksStoreQueryable(stores, finder, NewBlocksConsistencyChecker(0, 0, logger, nil), &blocksStoreLimitsMock{}, 0, logger, nil) - require.NoError(t, err) - require.NoError(t, services.StartAndAwaitRunning(context.Background(), queryable)) - defer services.StopAndAwaitTerminated(context.Background(), queryable) // nolint:errcheck + // Instantiate the querier that will be executed to run the query. + logger := log.NewNopLogger() + queryable, err := NewBlocksStoreQueryable(stores, finder, NewBlocksConsistencyChecker(0, 0, logger, nil), &blocksStoreLimitsMock{}, 0, 0, logger, nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), queryable)) + defer services.StopAndAwaitTerminated(context.Background(), queryable) // nolint:errcheck - engine := promql.NewEngine(promql.EngineOpts{ - Logger: logger, - Timeout: 10 * time.Second, - MaxSamples: 1e6, - }) + engine := promql.NewEngine(promql.EngineOpts{ + Logger: logger, + Timeout: 10 * time.Second, + MaxSamples: 1e6, + }) - // Query metrics. - ctx := user.InjectOrgID(context.Background(), "user-1") - q, err := engine.NewRangeQuery(ctx, queryable, nil, testData.query, queryStart, queryEnd, 15*time.Second) - require.NoError(t, err) + // Query metrics. + ctx := user.InjectOrgID(context.Background(), "user-1") + q, err := engine.NewRangeQuery(ctx, queryable, nil, testData.query, queryStart, queryEnd, 15*time.Second) + require.NoError(t, err) - res := q.Exec(ctx) - require.NoError(t, err) - require.NoError(t, res.Err) + res := q.Exec(ctx) + require.NoError(t, err) + require.NoError(t, res.Err) - matrix, err := res.Matrix() - require.NoError(t, err) - assert.Equal(t, testData.expected, matrix) + matrix, err := res.Matrix() + require.NoError(t, err) + assert.Equal(t, testData.expected, matrix) + }) + } }) } } @@ -1904,8 +2017,9 @@ type storeGatewayClientMock struct { mockedLabelValuesErr error } -func (m *storeGatewayClientMock) Series(context.Context, *storepb.SeriesRequest, ...grpc.CallOption) (storegatewaypb.StoreGateway_SeriesClient, error) { +func (m *storeGatewayClientMock) Series(ctx context.Context, _ *storepb.SeriesRequest, _ ...grpc.CallOption) (storegatewaypb.StoreGateway_SeriesClient, error) { seriesClient := &storeGatewaySeriesClientMock{ + ClientStream: grpcClientStreamMock{ctx: ctx}, // Required to not panic. mockedResponses: m.mockedSeriesResponses, } @@ -1943,6 +2057,17 @@ func (m *storeGatewaySeriesClientMock) Recv() (*storepb.SeriesResponse, error) { return res, nil } +type grpcClientStreamMock struct { + ctx context.Context +} + +func (grpcClientStreamMock) Header() (metadata.MD, error) { return nil, nil } +func (grpcClientStreamMock) Trailer() metadata.MD { return nil } +func (grpcClientStreamMock) CloseSend() error { return nil } +func (m grpcClientStreamMock) Context() context.Context { return m.ctx } +func (grpcClientStreamMock) SendMsg(interface{}) error { return nil } +func (grpcClientStreamMock) RecvMsg(interface{}) error { return nil } + type cancelerStoreGatewaySeriesClientMock struct { storeGatewaySeriesClientMock ctx context.Context @@ -1965,6 +2090,9 @@ func (m *cancelerStoreGatewayClientMock) Series(ctx context.Context, _ *storepb. series := &cancelerStoreGatewaySeriesClientMock{ ctx: ctx, cancel: m.cancel, + storeGatewaySeriesClientMock: storeGatewaySeriesClientMock{ + ClientStream: grpcClientStreamMock{ctx: ctx}, + }, } return series, nil } @@ -2035,6 +2163,34 @@ func mockSeriesResponseWithChunks(lbls labels.Labels, chunks ...storepb.AggrChun } } +func mockStreamingSeriesBatchResponse(endOfStream bool, lbls ...[]mimirpb.LabelAdapter) *storepb.SeriesResponse { + res := &storepb.StreamingSeriesBatch{} + for _, l := range lbls { + res.Series = append(res.Series, &storepb.StreamingSeries{Labels: l}) + } + res.IsEndOfSeriesStream = endOfStream + return &storepb.SeriesResponse{ + Result: &storepb.SeriesResponse_StreamingSeries{ + StreamingSeries: res, + }, + } +} + +func mockStreamingSeriesChunksResponse(index uint64, chks []storepb.AggrChunk) *storepb.SeriesResponse { + return &storepb.SeriesResponse{ + Result: &storepb.SeriesResponse_StreamingChunks{ + StreamingChunks: &storepb.StreamingChunksBatch{ + Series: []*storepb.StreamingChunks{ + { + SeriesIndex: index, + Chunks: chks, + }, + }, + }, + }, + } +} + func mockStatsResponse(fetchedIndexBytes int) *storepb.SeriesResponse { return &storepb.SeriesResponse{ Result: &storepb.SeriesResponse_Stats{ diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index f5308f9d390..b93c352ab4a 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -51,10 +51,12 @@ type Config struct { ShuffleShardingIngestersEnabled bool `yaml:"shuffle_sharding_ingesters_enabled" category:"advanced"` - PreferStreamingChunks bool `yaml:"prefer_streaming_chunks" category:"experimental"` - StreamingChunksPerIngesterSeriesBufferSize uint64 `yaml:"streaming_chunks_per_ingester_series_buffer_size" category:"experimental"` - MinimizeIngesterRequests bool `yaml:"minimize_ingester_requests" category:"experimental"` - MinimiseIngesterRequestsHedgingDelay time.Duration `yaml:"minimize_ingester_requests_hedging_delay" category:"experimental"` + PreferStreamingChunksFromIngesters bool `yaml:"prefer_streaming_chunks_from_ingesters" category:"experimental"` + PreferStreamingChunksFromStoreGateways bool `yaml:"prefer_streaming_chunks_from_store_gateways" category:"experimental"` + StreamingChunksPerIngesterSeriesBufferSize uint64 `yaml:"streaming_chunks_per_ingester_series_buffer_size" category:"experimental"` + StreamingChunksPerStoreGatewaySeriesBufferSize uint64 `yaml:"streaming_chunks_per_store_gateway_series_buffer_size" category:"experimental"` + MinimizeIngesterRequests bool `yaml:"minimize_ingester_requests" category:"experimental"` + MinimiseIngesterRequestsHedgingDelay time.Duration `yaml:"minimize_ingester_requests_hedging_delay" category:"experimental"` // PromQL engine config. EngineConfig engine.Config `yaml:",inline"` @@ -83,15 +85,17 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.MaxQueryIntoFuture, "querier.max-query-into-future", 10*time.Minute, "Maximum duration into the future you can query. 0 to disable.") f.DurationVar(&cfg.QueryStoreAfter, queryStoreAfterFlag, 12*time.Hour, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. If this option is enabled, the time range of the query sent to the store-gateway will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") f.BoolVar(&cfg.ShuffleShardingIngestersEnabled, "querier.shuffle-sharding-ingesters-enabled", true, fmt.Sprintf("Fetch in-memory series from the minimum set of required ingesters, selecting only ingesters which may have received series since -%s. If this setting is false or -%s is '0', queriers always query all ingesters (ingesters shuffle sharding on read path is disabled).", validation.QueryIngestersWithinFlag, validation.QueryIngestersWithinFlag)) - f.BoolVar(&cfg.PreferStreamingChunks, "querier.prefer-streaming-chunks", false, "Request ingesters stream chunks. Ingesters will only respond with a stream of chunks if the target ingester supports this, and this preference will be ignored by ingesters that do not support this.") + f.BoolVar(&cfg.PreferStreamingChunksFromIngesters, "querier.prefer-streaming-chunks-from-ingesters", false, "Request ingesters stream chunks. Ingesters will only respond with a stream of chunks if the target ingester supports this, and this preference will be ignored by ingesters that do not support this.") + f.BoolVar(&cfg.PreferStreamingChunksFromStoreGateways, "querier.prefer-streaming-chunks-from-store-gateways", false, "Request store-gateways stream chunks. Store-gateways will only respond with a stream of chunks if the target store-gateway supports this, and this preference will be ignored by store-gateways that do not support this.") const minimiseIngesterRequestsFlagName = "querier.minimize-ingester-requests" f.BoolVar(&cfg.MinimizeIngesterRequests, minimiseIngesterRequestsFlagName, false, "If true, when querying ingesters, only the minimum required ingesters required to reach quorum will be queried initially, with other ingesters queried only if needed due to failures from the initial set of ingesters. Enabling this option reduces resource consumption for the happy path at the cost of increased latency for the unhappy path.") f.DurationVar(&cfg.MinimiseIngesterRequestsHedgingDelay, minimiseIngesterRequestsFlagName+"-hedging-delay", 3*time.Second, "Delay before initiating requests to further ingesters when request minimization is enabled and the initially selected set of ingesters have not all responded. Ignored if -"+minimiseIngesterRequestsFlagName+" is not enabled.") - // Why 256 series / ingester? + // Why 256 series / ingester/store-gateway? // Based on our testing, 256 series / ingester was a good balance between memory consumption and the CPU overhead of managing a batch of series. f.Uint64Var(&cfg.StreamingChunksPerIngesterSeriesBufferSize, "querier.streaming-chunks-per-ingester-buffer-size", 256, "Number of series to buffer per ingester when streaming chunks from ingesters.") + f.Uint64Var(&cfg.StreamingChunksPerStoreGatewaySeriesBufferSize, "querier.streaming-chunks-per-store-gateway-buffer-size", 256, "Number of series to buffer per store-gateway when streaming chunks from store-gateways.") // The querier.query-ingesters-within flag has been moved to the limits.go file // We still need to set a default value for cfg.QueryIngestersWithin since we need to keep supporting the querier yaml field until Mimir 2.11.0 diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index de52bcbc08a..5b539e1d642 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -68,6 +68,8 @@ const ( // Labels for metrics. labelEncode = "encode" labelDecode = "decode" + + targetQueryStreamBatchMessageSize = 1 * 1024 * 1024 ) type BucketStoreStats struct { @@ -543,6 +545,10 @@ type seriesChunks struct { // Series implements the storepb.StoreServer interface. func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_SeriesServer) (err error) { + if req.SkipChunks { + // We don't do the streaming call if we are not requesting the chunks. + req.StreamingChunksBatchSize = 0 + } defer func() { if err == nil { return @@ -579,11 +585,10 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } var ( + spanLogger = spanlogger.FromContext(srv.Context(), s.logger) ctx = srv.Context() stats = newSafeQueryStats() reqBlockMatchers []*labels.Matcher - chunksLimiter = s.chunksLimiterFactory(s.metrics.queriesDropped.WithLabelValues("chunks")) - seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series")) ) defer s.recordSeriesCallResult(stats) @@ -599,7 +604,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } } - logSeriesRequestToSpan(srv.Context(), s.logger, req.MinTime, req.MaxTime, matchers, reqBlockMatchers, shardSelector) + logSeriesRequestToSpan(srv.Context(), s.logger, req.MinTime, req.MaxTime, matchers, reqBlockMatchers, shardSelector, req.StreamingChunksBatchSize) blocks, indexReaders, chunkReaders := s.openBlocksForReading(ctx, req.SkipChunks, req.MinTime, req.MaxTime, reqBlockMatchers, stats) // We must keep the readers open until all their data has been sent. @@ -615,105 +620,336 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie readers = newChunkReaders(chunkReaders) } - seriesSet, resHints, err := s.streamingSeriesSetForBlocks(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) - if err != nil { + var ( + // If we are streaming the series labels and chunks separately, we don't need to fetch the postings + // twice. So we use these slices to re-use them. Each reuse[i] corresponds to a single block. + reuse []*reusedPostingsAndMatchers + resHints = &hintspb.SeriesResponseHints{} + ) + for _, b := range blocks { + resHints.AddQueriedBlock(b.meta.ULID) + } + if err := s.sendHints(srv, resHints); err != nil { return err } - - // Merge the sub-results from each selected block. - tracing.DoWithSpan(ctx, "bucket_store_merge_all", func(ctx context.Context, _ tracing.Span) { + if req.StreamingChunksBatchSize > 0 { var ( - iterationBegin = time.Now() - encodeDuration = time.Duration(0) - sendDuration = time.Duration(0) - seriesCount int - chunksCount int + seriesSet storepb.SeriesSet + seriesLoadStart = time.Now() + chunksLimiter = s.chunksLimiterFactory(s.metrics.queriesDropped.WithLabelValues("chunks")) + seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series")) ) - // Once the iteration is done we will update the stats. - defer stats.update(func(stats *queryStats) { - stats.mergedSeriesCount += seriesCount - stats.mergedChunksCount += chunksCount - - // The time spent iterating over the series set is the - // actual time spent fetching series and chunks, encoding and sending them to the client. - // We split the timings to have a better view over how time is spent. - stats.streamingSeriesFetchSeriesAndChunksDuration += stats.streamingSeriesWaitBatchLoadedDuration - stats.streamingSeriesEncodeResponseDuration += encodeDuration - stats.streamingSeriesSendResponseDuration += sendDuration - stats.streamingSeriesOtherDuration += time.Duration(util_math.Max(0, int64(time.Since(iterationBegin)- - stats.streamingSeriesFetchSeriesAndChunksDuration-encodeDuration-sendDuration))) - }) + seriesSet, reuse, err = s.streamingSeriesForBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) + if err != nil { + return err + } - for seriesSet.Next() { - var lset labels.Labels - var series storepb.Series + numSeries, err := s.sendStreamingSeriesLabelsAndStats(req, srv, stats, seriesSet) + if err != nil { + return err + } + level.Debug(spanLogger).Log( + "msg", "sent streaming series", + "num_series", numSeries, + "duration", time.Since(seriesLoadStart), + ) - seriesCount++ + if numSeries == 0 { + // There is no series to send chunks for. + return nil + } + } - // IMPORTANT: do not retain the memory returned by seriesSet.At() beyond this loop cycle - // because the subsequent call to seriesSet.Next() may release it. - if req.SkipChunks { - lset, _ = seriesSet.At() - } else { - lset, series.Chunks = seriesSet.At() + // We create the limiter twice in the case of streaming so that we don't double count the series + // and hit the limit prematurely. + chunksLimiter := s.chunksLimiterFactory(s.metrics.queriesDropped.WithLabelValues("chunks")) + seriesLimiter := s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series")) - chunksCount += len(series.Chunks) - s.metrics.chunkSizeBytes.Observe(float64(chunksSize(series.Chunks))) - } - series.Labels = mimirpb.FromLabelsToLabelAdapters(lset) - - // Encode the message. We encode it ourselves into a PreparedMsg in order to measure - // the time it takes. - encodeBegin := time.Now() - msg := &grpc.PreparedMsg{} - if err = msg.Encode(srv, storepb.NewSeriesResponse(&series)); err != nil { - err = status.Error(codes.Internal, errors.Wrap(err, "encode series response").Error()) - return + start := time.Now() + if req.StreamingChunksBatchSize > 0 { + var seriesChunkIt seriesChunksSetIterator + seriesChunkIt, err = s.streamingChunksSetForBlocks(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse) + if err != nil { + return err + } + err = s.sendStreamingChunks(req, srv, seriesChunkIt, stats) + } else { + var seriesSet storepb.SeriesSet + seriesSet, err = s.nonStreamingSeriesSetForBlocks(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) + if err != nil { + return err + } + err = s.sendSeriesChunks(req, srv, seriesSet, stats) + } + if err != nil { + return + } + + numSeries, numChunks := stats.seriesAndChunksCount() + debugMessage := "sent series" + if req.StreamingChunksBatchSize > 0 { + debugMessage = "sent streaming chunks" + } + level.Debug(spanLogger).Log( + "msg", debugMessage, + "num_series", numSeries, + "num_chunks", numChunks, + "duration", time.Since(start), + ) + + if req.StreamingChunksBatchSize == 0 { + // Stats were not sent before, so send it now. + return s.sendStats(srv, stats) + } + + return nil +} + +// sendStreamingSeriesLabelsAndStats sends the labels of the streaming series. +// Since hints and stats need to be sent before the "end of stream" streaming series message, +// this function also sends the hints and the stats. +func (s *BucketStore) sendStreamingSeriesLabelsAndStats( + req *storepb.SeriesRequest, + srv storepb.Store_SeriesServer, + stats *safeQueryStats, + seriesSet storepb.SeriesSet, +) (numSeries int, err error) { + var ( + encodeDuration = time.Duration(0) + sendDuration = time.Duration(0) + iterationBegin = time.Now() + ) + defer stats.update(func(stats *queryStats) { + // The time spent iterating over the series set is the + // actual time spent fetching series and chunks, encoding and sending them to the client. + // We split the timings to have a better view over how time is spent. + // We do not update streamingSeriesFetchSeriesAndChunksDuration here because it will be updated when sending + // streaming chunks, that includes the series and chunks fetch duration for sending the streaming series. + stats.streamingSeriesEncodeResponseDuration += encodeDuration + stats.streamingSeriesSendResponseDuration += sendDuration + stats.streamingSeriesOtherDuration += time.Duration(util_math.Max(0, int64(time.Since(iterationBegin)- + stats.streamingSeriesFetchSeriesAndChunksDuration-encodeDuration-sendDuration))) + }) + + seriesBuffer := make([]*storepb.StreamingSeries, req.StreamingChunksBatchSize) + for i := range seriesBuffer { + seriesBuffer[i] = &storepb.StreamingSeries{} + } + seriesBatch := &storepb.StreamingSeriesBatch{ + Series: seriesBuffer[:0], + } + // TODO: can we send this in parallel while we start fetching the chunks below? + for seriesSet.Next() { + numSeries++ + var lset labels.Labels + // Although subsequent call to seriesSet.Next() may release the memory of this series object, + // it is safe to hold onto the labels because they are not released. + lset, _ = seriesSet.At() + + // We are re-using the slice for every batch this way. + seriesBatch.Series = seriesBatch.Series[:len(seriesBatch.Series)+1] + seriesBatch.Series[len(seriesBatch.Series)-1].Labels = mimirpb.FromLabelsToLabelAdapters(lset) + + if len(seriesBatch.Series) == int(req.StreamingChunksBatchSize) { + err := s.sendMessage("streaming series", srv, storepb.NewStreamingSeriesResponse(seriesBatch), &encodeDuration, &sendDuration) + if err != nil { + return 0, err } - encodeDuration += time.Since(encodeBegin) + seriesBatch.Series = seriesBatch.Series[:0] + } + } + if seriesSet.Err() != nil { + return 0, errors.Wrap(seriesSet.Err(), "expand series set") + } + + // We need to send stats before sending IsEndOfSeriesStream=true. + if err := s.sendStats(srv, stats); err != nil { + return 0, err + } + + // Send any remaining series and signal that there are no more series. + seriesBatch.IsEndOfSeriesStream = true + err = s.sendMessage("streaming series", srv, storepb.NewStreamingSeriesResponse(seriesBatch), &encodeDuration, &sendDuration) + return numSeries, err +} - // Send the message. - sendBegin := time.Now() - if err = srv.SendMsg(msg); err != nil { - err = status.Error(codes.Unknown, errors.Wrap(err, "send series response").Error()) - return +func (s *BucketStore) sendStreamingChunks( + req *storepb.SeriesRequest, + srv storepb.Store_SeriesServer, + it seriesChunksSetIterator, + stats *safeQueryStats, +) error { + var ( + encodeDuration time.Duration + sendDuration time.Duration + seriesCount, chunksCount int + iterationBegin = time.Now() + ) + + defer stats.update(func(stats *queryStats) { + stats.mergedSeriesCount += seriesCount + stats.mergedChunksCount += chunksCount + + // The time spent iterating over the series set is the + // actual time spent fetching series and chunks, encoding and sending them to the client. + // We split the timings to have a better view over how time is spent. + stats.streamingSeriesFetchSeriesAndChunksDuration += stats.streamingSeriesWaitBatchLoadedDuration + stats.streamingSeriesEncodeResponseDuration += encodeDuration + stats.streamingSeriesSendResponseDuration += sendDuration + stats.streamingSeriesOtherDuration += time.Duration(util_math.Max(0, int64(time.Since(iterationBegin)- + stats.streamingSeriesFetchSeriesAndChunksDuration-encodeDuration-sendDuration))) + }) + + var ( + batchSizeBytes int + chunksBuffer = make([]*storepb.StreamingChunks, req.StreamingChunksBatchSize) + ) + for i := range chunksBuffer { + chunksBuffer[i] = &storepb.StreamingChunks{} + } + chunksBatch := &storepb.StreamingChunksBatch{Series: chunksBuffer[:0]} + for it.Next() { + set := it.At() + for _, sc := range set.series { + seriesCount++ + chunksBatch.Series = chunksBatch.Series[:len(chunksBatch.Series)+1] + lastSeries := chunksBatch.Series[len(chunksBatch.Series)-1] + lastSeries.Chunks = sc.chks + lastSeries.SeriesIndex = uint64(seriesCount - 1) + + batchSizeBytes += lastSeries.Size() + + chunksCount += len(sc.chks) + s.metrics.chunkSizeBytes.Observe(float64(chunksSize(sc.chks))) + + // We are not strictly required to be under targetQueryStreamBatchMessageSize. + // The aim is to not hit gRPC and TCP limits, hence some overage is ok. + if batchSizeBytes > targetQueryStreamBatchMessageSize || len(chunksBatch.Series) >= int(req.StreamingChunksBatchSize) { + err := s.sendMessage("streaming chunks", srv, storepb.NewStreamingChunksResponse(chunksBatch), &encodeDuration, &sendDuration) + if err != nil { + return err + } + chunksBatch.Series = chunksBatch.Series[:0] + batchSizeBytes = 0 } - sendDuration += time.Since(sendBegin) } - if seriesSet.Err() != nil { - err = errors.Wrap(seriesSet.Err(), "expand series set") - return + + if len(chunksBatch.Series) > 0 { + // Still some chunks left to send before we release the batch. + err := s.sendMessage("streaming chunks", srv, storepb.NewStreamingChunksResponse(chunksBatch), &encodeDuration, &sendDuration) + if err != nil { + return err + } + chunksBatch.Series = chunksBatch.Series[:0] + batchSizeBytes = 0 } - err = nil + set.release() + } + + if it.Err() != nil { + return it.Err() + } + + return it.Err() +} + +func (s *BucketStore) sendSeriesChunks( + req *storepb.SeriesRequest, + srv storepb.Store_SeriesServer, + seriesSet storepb.SeriesSet, + stats *safeQueryStats, +) error { + var ( + encodeDuration time.Duration + sendDuration time.Duration + seriesCount, chunksCount int + iterationBegin = time.Now() + ) + + defer stats.update(func(stats *queryStats) { + stats.mergedSeriesCount += seriesCount + stats.mergedChunksCount += chunksCount + + // The time spent iterating over the series set is the + // actual time spent fetching series and chunks, encoding and sending them to the client. + // We split the timings to have a better view over how time is spent. + stats.streamingSeriesFetchSeriesAndChunksDuration += stats.streamingSeriesWaitBatchLoadedDuration + stats.streamingSeriesEncodeResponseDuration += encodeDuration + stats.streamingSeriesSendResponseDuration += sendDuration + stats.streamingSeriesOtherDuration += time.Duration(util_math.Max(0, int64(time.Since(iterationBegin)- + stats.streamingSeriesFetchSeriesAndChunksDuration-encodeDuration-sendDuration))) }) - if err != nil { - return + for seriesSet.Next() { + seriesCount++ + // IMPORTANT: do not retain the memory returned by seriesSet.At() beyond this loop cycle + // because the subsequent call to seriesSet.Next() may release it. But it is safe to hold + // onto lset because the labels are not released. + lset, chks := seriesSet.At() + series := storepb.Series{ + Labels: mimirpb.FromLabelsToLabelAdapters(lset), + } + if !req.SkipChunks { + series.Chunks = chks + chunksCount += len(chks) + s.metrics.chunkSizeBytes.Observe(float64(chunksSize(chks))) + } + + err := s.sendMessage("series", srv, storepb.NewSeriesResponse(&series), &encodeDuration, &sendDuration) + if err != nil { + return err + } } + if seriesSet.Err() != nil { + return errors.Wrap(seriesSet.Err(), "expand series set") + } + + return nil +} + +func (s *BucketStore) sendMessage(typ string, srv storepb.Store_SeriesServer, msg interface{}, encodeDuration, sendDuration *time.Duration) error { + // We encode it ourselves into a PreparedMsg in order to measure the time it takes. + encodeBegin := time.Now() + pmsg := &grpc.PreparedMsg{} + if err := pmsg.Encode(srv, msg); err != nil { + return status.Error(codes.Internal, errors.Wrapf(err, "encode %s response", typ).Error()) + } + *encodeDuration += time.Since(encodeBegin) + sendBegin := time.Now() + if err := srv.SendMsg(pmsg); err != nil { + return status.Error(codes.Unknown, errors.Wrapf(err, "send %s response", typ).Error()) + } + *sendDuration += time.Since(sendBegin) + + return nil +} + +func (s *BucketStore) sendHints(srv storepb.Store_SeriesServer, resHints *hintspb.SeriesResponseHints) error { var anyHints *types.Any + var err error if anyHints, err = types.MarshalAny(resHints); err != nil { - err = status.Error(codes.Unknown, errors.Wrap(err, "marshal series response hints").Error()) - return + return status.Error(codes.Internal, errors.Wrap(err, "marshal series response hints").Error()) } - if err = srv.Send(storepb.NewHintsSeriesResponse(anyHints)); err != nil { - err = status.Error(codes.Unknown, errors.Wrap(err, "send series response hints").Error()) - return + if err := srv.Send(storepb.NewHintsSeriesResponse(anyHints)); err != nil { + return status.Error(codes.Unknown, errors.Wrap(err, "send series response hints").Error()) } + return nil +} + +func (s *BucketStore) sendStats(srv storepb.Store_SeriesServer, stats *safeQueryStats) error { unsafeStats := stats.export() - if err = srv.Send(storepb.NewStatsResponse(unsafeStats.postingsTouchedSizeSum + unsafeStats.seriesProcessedSizeSum)); err != nil { - err = status.Error(codes.Unknown, errors.Wrap(err, "sends series response stats").Error()) - return + if err := srv.Send(storepb.NewStatsResponse(unsafeStats.postingsTouchedSizeSum + unsafeStats.seriesProcessedSizeSum)); err != nil { + return status.Error(codes.Unknown, errors.Wrap(err, "sends series response stats").Error()) } - - return err + return nil } -func logSeriesRequestToSpan(ctx context.Context, l log.Logger, minT, maxT int64, matchers, blockMatchers []*labels.Matcher, shardSelector *sharding.ShardSelector) { +func logSeriesRequestToSpan(ctx context.Context, l log.Logger, minT, maxT int64, matchers, blockMatchers []*labels.Matcher, shardSelector *sharding.ShardSelector, streamingChunksBatchSize uint64) { spanLogger := spanlogger.FromContext(ctx, l) level.Debug(spanLogger).Log( "msg", "BucketStore.Series", @@ -722,6 +958,7 @@ func logSeriesRequestToSpan(ctx context.Context, l log.Logger, minT, maxT int64, "request matchers", storepb.PromMatchersToString(matchers...), "request block matchers", storepb.PromMatchersToString(blockMatchers...), "request shard selector", maybeNilShard(shardSelector).LabelValue(), + "streaming chunks batch size", streamingChunksBatchSize, ) } @@ -732,7 +969,8 @@ func chunksSize(chks []storepb.AggrChunk) (size int) { return size } -func (s *BucketStore) streamingSeriesSetForBlocks( +// nonStreamingSeriesSetForBlocks is used when the streaming feature is not enabled. +func (s *BucketStore) nonStreamingSeriesSetForBlocks( ctx context.Context, req *storepb.SeriesRequest, blocks []*bucketBlock, @@ -743,20 +981,111 @@ func (s *BucketStore) streamingSeriesSetForBlocks( chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, -) (storepb.SeriesSet, *hintspb.SeriesResponseHints, error) { +) (storepb.SeriesSet, error) { + var strategy seriesIteratorStrategy + if req.SkipChunks { + strategy = noChunkRefs + } + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, nil, strategy) + if err != nil { + return nil, err + } + + var set storepb.SeriesSet + if !req.SkipChunks { + var cache chunkscache.Cache + if s.fineGrainedChunksCachingEnabled { + cache = s.chunksCache + } + ss := newChunksPreloadingIterator(ctx, s.logger, s.userID, cache, *chunkReaders, it, s.maxSeriesPerBatch, stats, req.MinTime, req.MaxTime) + set = newSeriesChunksSeriesSet(ss) + } else { + set = newSeriesSetWithoutChunks(ctx, it, stats) + } + return set, nil +} + +// streamingSeriesForBlocks is used when streaming feature is enabled. +// It returns a series set that only contains the series labels without any chunks information. +// The returned postings (series ref) and matches should be re-used when getting chunks to save on computation. +func (s *BucketStore) streamingSeriesForBlocks( + ctx context.Context, + req *storepb.SeriesRequest, + blocks []*bucketBlock, + indexReaders map[ulid.ULID]*bucketIndexReader, + shardSelector *sharding.ShardSelector, + matchers []*labels.Matcher, + chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. + seriesLimiter SeriesLimiter, // Rate limiter for loading series. + stats *safeQueryStats, +) (storepb.SeriesSet, []*reusedPostingsAndMatchers, error) { var ( - resHints = &hintspb.SeriesResponseHints{} - mtx = sync.Mutex{} - batches = make([]seriesChunkRefsSetIterator, 0, len(blocks)) - g, _ = errgroup.WithContext(ctx) - begin = time.Now() + reuse = make([]*reusedPostingsAndMatchers, len(blocks)) + strategy = noChunkRefs | overlapMintMaxt ) + for i := range reuse { + reuse[i] = &reusedPostingsAndMatchers{} + } + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse, strategy) + if err != nil { + return nil, nil, err + } + return newSeriesSetWithoutChunks(ctx, it, stats), reuse, nil +} - for _, b := range blocks { +// streamingChunksSetForBlocks is used when streaming feature is enabled. +// It returns an iterator to go over the chunks for the series returned in the streamingSeriesForBlocks call. +// It is recommended to pass the reusePostings and reusePendingMatches returned by the streamingSeriesForBlocks call. +func (s *BucketStore) streamingChunksSetForBlocks( + ctx context.Context, + req *storepb.SeriesRequest, + blocks []*bucketBlock, + indexReaders map[ulid.ULID]*bucketIndexReader, + chunkReaders *bucketChunkReaders, + shardSelector *sharding.ShardSelector, + matchers []*labels.Matcher, + chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. + seriesLimiter SeriesLimiter, // Rate limiter for loading series. + stats *safeQueryStats, + reuse []*reusedPostingsAndMatchers, // Should come from streamingSeriesForBlocks. +) (seriesChunksSetIterator, error) { + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse, defaultStrategy) + if err != nil { + return nil, err + } + + var cache chunkscache.Cache + if s.fineGrainedChunksCachingEnabled { + cache = s.chunksCache + } + scsi := newChunksPreloadingIterator(ctx, s.logger, s.userID, cache, *chunkReaders, it, s.maxSeriesPerBatch, stats, req.MinTime, req.MaxTime) + return scsi, nil +} + +func (s *BucketStore) getSeriesIteratorFromBlocks( + ctx context.Context, + req *storepb.SeriesRequest, + blocks []*bucketBlock, + indexReaders map[ulid.ULID]*bucketIndexReader, + shardSelector *sharding.ShardSelector, + matchers []*labels.Matcher, + chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. + seriesLimiter SeriesLimiter, // Rate limiter for loading series. + stats *safeQueryStats, + reuse []*reusedPostingsAndMatchers, // Used if not empty. If not empty, len(reuse) must be len(blocks). + strategy seriesIteratorStrategy, +) (seriesChunkRefsSetIterator, error) { + var ( + mtx = sync.Mutex{} + batches = make([]seriesChunkRefsSetIterator, 0, len(blocks)) + g, _ = errgroup.WithContext(ctx) + begin = time.Now() + ) + for i, b := range blocks { b := b + i := i // Keep track of queried blocks. - resHints.AddQueriedBlock(b.meta.ULID) indexr := indexReaders[b.meta.ULID] // If query sharding is enabled we have to get the block-specific series hash cache @@ -765,13 +1094,12 @@ func (s *BucketStore) streamingSeriesSetForBlocks( if shardSelector != nil { blockSeriesHashCache = s.seriesHashCache.GetBlockCache(b.meta.ULID.String()) } + var r *reusedPostingsAndMatchers + if len(reuse) > 0 { + r = reuse[i] + } g.Go(func() error { - var ( - part seriesChunkRefsSetIterator - err error - ) - - part, err = openBlockSeriesChunkRefsSetsIterator( + part, err := openBlockSeriesChunkRefsSetsIterator( ctx, s.maxSeriesPerBatch, s.userID, @@ -781,10 +1109,11 @@ func (s *BucketStore) streamingSeriesSetForBlocks( matchers, shardSelector, cachedSeriesHasher{blockSeriesHashCache}, - req.SkipChunks, + strategy, req.MinTime, req.MaxTime, s.numChunksRangesPerSeries, stats, + r, s.logger, ) if err != nil { @@ -801,7 +1130,7 @@ func (s *BucketStore) streamingSeriesSetForBlocks( err := g.Wait() if err != nil { - return nil, nil, err + return nil, err } stats.update(func(stats *queryStats) { @@ -815,17 +1144,7 @@ func (s *BucketStore) streamingSeriesSetForBlocks( // counted once towards the limit. mergedIterator = newLimitingSeriesChunkRefsSetIterator(mergedIterator, chunksLimiter, seriesLimiter) - var set storepb.SeriesSet - if !req.SkipChunks { - var cache chunkscache.Cache - if s.fineGrainedChunksCachingEnabled { - cache = s.chunksCache - } - set = newSeriesSetWithChunks(ctx, s.logger, s.userID, cache, *chunkReaders, mergedIterator, s.maxSeriesPerBatch, stats, req.MinTime, req.MaxTime) - } else { - set = newSeriesSetWithoutChunks(ctx, mergedIterator, stats) - } - return set, resHints, nil + return mergedIterator, nil } func (s *BucketStore) recordSeriesCallResult(safeStats *safeQueryStats) { @@ -1080,10 +1399,11 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] matchers, nil, cachedSeriesHasher{nil}, - true, + noChunkRefs, minTime, maxTime, 1, // we skip chunks, so this doesn't make any difference stats, + nil, logger, ) if err != nil { @@ -1299,7 +1619,7 @@ func labelValuesFromSeries(ctx context.Context, labelName string, seriesPerBatch b.meta, nil, nil, - true, + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, b.userID, @@ -1656,7 +1976,7 @@ type symbolizedLabel struct { // decodeSeries decodes a series entry from the given byte slice decoding all chunk metas of the series. // If skipChunks is specified decodeSeries does not return any chunks, but only labels and only if at least single chunk is within time range. // decodeSeries returns false, when there are no series data for given time range. -func decodeSeries(b []byte, lsetPool *pool.SlabPool[symbolizedLabel], chks *[]chunks.Meta, skipChunks bool) (ok bool, lset []symbolizedLabel, err error) { +func decodeSeries(b []byte, lsetPool *pool.SlabPool[symbolizedLabel], chks *[]chunks.Meta, strategy seriesIteratorStrategy) (ok bool, lset []symbolizedLabel, err error) { *chks = (*chks)[:0] @@ -1682,6 +2002,7 @@ func decodeSeries(b []byte, lsetPool *pool.SlabPool[symbolizedLabel], chks *[]ch // Similar for first ref. ref := int64(d.Uvarint64()) + isNoChunks := strategy.isNoChunkRefsOnEntireBlock() for i := 0; i < k; i++ { if i > 0 { mint += int64(d.Uvarint64()) @@ -1690,7 +2011,7 @@ func decodeSeries(b []byte, lsetPool *pool.SlabPool[symbolizedLabel], chks *[]ch } // Found a chunk. - if skipChunks { + if isNoChunks { // We are not interested in chunks and we know there is at least one, that's enough to return series. return true, lset, nil } diff --git a/pkg/storegateway/bucket_chunk_reader_test.go b/pkg/storegateway/bucket_chunk_reader_test.go index 14d637fcbeb..186a136d197 100644 --- a/pkg/storegateway/bucket_chunk_reader_test.go +++ b/pkg/storegateway/bucket_chunk_reader_test.go @@ -38,11 +38,12 @@ func TestBucketChunkReader_refetchChunks(t *testing.T) { []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "j", "foo")}, nil, nil, - false, + defaultStrategy, block.meta.MinTime, block.meta.MaxTime, 2, newSafeQueryStats(), + nil, log.NewNopLogger(), ) require.NoError(t, err) diff --git a/pkg/storegateway/bucket_e2e_test.go b/pkg/storegateway/bucket_e2e_test.go index 55a14b09c84..a3b28f104ba 100644 --- a/pkg/storegateway/bucket_e2e_test.go +++ b/pkg/storegateway/bucket_e2e_test.go @@ -66,8 +66,11 @@ type storeSuite struct { logger log.Logger } +// When nonOverlappingBlocks is false, prepareTestBlocks creates 2 blocks per block range. +// When nonOverlappingBlocks is true, it shifts the 2nd block ahead by 2hrs for every block range. +// This way the first and the last blocks created have no overlapping blocks. func prepareTestBlocks(t testing.TB, now time.Time, count int, dir string, bkt objstore.Bucket, - series []labels.Labels, extLset labels.Labels) (minTime, maxTime int64) { + series []labels.Labels, extLset labels.Labels, nonOverlappingBlocks bool) (minTime, maxTime int64) { ctx := context.Background() logger := log.NewNopLogger() @@ -85,6 +88,11 @@ func prepareTestBlocks(t testing.TB, now time.Time, count int, dir string, bkt o // gets created each. This way we can easily verify we got 10 chunks per series below. id1, err := block.CreateBlock(ctx, dir, series[:4], 10, mint, maxt, extLset) assert.NoError(t, err) + if nonOverlappingBlocks { + mint = maxt + maxt = timestamp.FromTime(now.Add(2 * time.Hour)) + maxTime = maxt + } id2, err := block.CreateBlock(ctx, dir, series[4:], 10, mint, maxt, extLset) assert.NoError(t, err) @@ -117,6 +125,10 @@ type prepareStoreConfig struct { chunksCache chunkscache.Cache metricsRegistry *prometheus.Registry postingsStrategy postingsSelectionStrategy + // When nonOverlappingBlocks is false, prepare store creates 2 blocks per block range. + // When nonOverlappingBlocks is true, it shifts the 2nd block ahead by 2hrs for every block range. + // This way the first and the last blocks created have no overlapping blocks. + nonOverlappingBlocks bool } func (c *prepareStoreConfig) apply(opts ...prepareStoreConfigOption) *prepareStoreConfig { @@ -164,7 +176,7 @@ func withManyParts() prepareStoreConfigOption { func prepareStoreWithTestBlocks(t testing.TB, bkt objstore.Bucket, cfg *prepareStoreConfig) *storeSuite { extLset := labels.FromStrings("ext1", "value1") - minTime, maxTime := prepareTestBlocks(t, time.Now(), 3, cfg.tempDir, bkt, cfg.series, extLset) + minTime, maxTime := prepareTestBlocks(t, time.Now(), 3, cfg.tempDir, bkt, cfg.series, extLset, cfg.nonOverlappingBlocks) s := &storeSuite{ logger: log.NewNopLogger(), @@ -218,10 +230,16 @@ func prepareStoreWithTestBlocks(t testing.TB, bkt objstore.Bucket, cfg *prepareS return s } +type testBucketStoreCase struct { + req *storepb.SeriesRequest + expected [][]mimirpb.LabelAdapter + expectedChunkLen int +} + // TODO(bwplotka): Benchmark Series. // //nolint:revive -func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { +func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite, additionalCases ...testBucketStoreCase) { t.Helper() mint, maxt := s.store.TimeRange() @@ -239,11 +257,7 @@ func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { srv := newBucketStoreTestServer(t, s.store) // TODO(bwplotka): Add those test cases to TSDB querier_test.go as well, there are no tests for matching. - for i, tcase := range []struct { - req *storepb.SeriesRequest - expected [][]mimirpb.LabelAdapter - expectedChunkLen int - }{ + testCases := []testBucketStoreCase{ { req: &storepb.SeriesRequest{ Matchers: []storepb.LabelMatcher{ @@ -416,20 +430,24 @@ func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { {{Name: "a", Value: "1"}, {Name: "c", Value: "2"}}, }, }, - } { - if ok := t.Run(fmt.Sprint(i), func(t *testing.T) { - seriesSet, _, _, err := srv.Series(context.Background(), tcase.req) - require.NoError(t, err) - - assert.Equal(t, len(tcase.expected), len(seriesSet)) - - for i, s := range seriesSet { - assert.Equal(t, tcase.expected[i], s.Labels) - assert.Equal(t, tcase.expectedChunkLen, len(s.Chunks)) + } + for i, tcase := range append(testCases, additionalCases...) { + for _, streamingBatchSize := range []int{0, 1, 5, 256} { + if ok := t.Run(fmt.Sprintf("%d,streamingBatchSize=%d", i, streamingBatchSize), func(t *testing.T) { + tcase.req.StreamingChunksBatchSize = uint64(streamingBatchSize) + seriesSet, _, _, err := srv.Series(context.Background(), tcase.req) + require.NoError(t, err) + + assert.Equal(t, len(tcase.expected), len(seriesSet)) + + for i, s := range seriesSet { + assert.Equal(t, tcase.expected[i], s.Labels) + assert.Equal(t, tcase.expectedChunkLen, len(s.Chunks)) + } + assertQueryStatsMetricsRecorded(t, len(tcase.expected), tcase.expectedChunkLen, s.metricsRegistry) + }); !ok { + return } - assertQueryStatsMetricsRecorded(t, len(tcase.expected), tcase.expectedChunkLen, s.metricsRegistry) - }); !ok { - return } } } @@ -494,7 +512,85 @@ func TestBucketStore_e2e(t *testing.T) { return } - t.Run("with small index cache", func(t *testing.T) { + if ok := t.Run("with small index cache", func(t *testing.T) { + indexCache2, err := indexcache.NewInMemoryIndexCacheWithConfig(s.logger, nil, indexcache.InMemoryIndexCacheConfig{ + MaxItemSize: 50, + MaxSize: 100, + }) + assert.NoError(t, err) + s.cache.SwapIndexCacheWith(indexCache2) + testBucketStore_e2e(t, ctx, s) + }); !ok { + return + } + + t.Run("with large, sufficient index cache, and chunks cache", func(t *testing.T) { + indexCache, err := indexcache.NewInMemoryIndexCacheWithConfig(s.logger, nil, indexcache.InMemoryIndexCacheConfig{ + MaxItemSize: 1e5, + MaxSize: 2e5, + }) + assert.NoError(t, err) + assert.NoError(t, err) + s.cache.SwapIndexCacheWith(indexCache) + s.cache.SwapChunksCacheWith(newInMemoryChunksCache()) + testBucketStore_e2e(t, ctx, s) + }) + }) +} + +func TestBucketStore_e2e_StreamingEdgeCases(t *testing.T) { + foreachStore(t, func(t *testing.T, newSuite suiteFactory) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + s := newSuite(func(config *prepareStoreConfig) { + config.nonOverlappingBlocks = true + }) + + _, maxt := s.store.TimeRange() + additionalCases := []testBucketStoreCase{ + { // This tests if the first phase of streaming that sends only the series is filtering the series by chunk time range. + // The request time range overlaps with 2 blocks with 4 timeseries each, but only the 2nd block + // has some overlapping data that should be returned. + req: &storepb.SeriesRequest{ + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "a", Value: "1|2"}, + }, + // A block spans 120 mins. So 121 grabs the second to last block. + MinTime: maxt - 121*int64(time.Minute/time.Millisecond), + MaxTime: maxt, + }, + expectedChunkLen: 1, + expected: [][]mimirpb.LabelAdapter{ + {{Name: "a", Value: "1"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "1"}, {Name: "c", Value: "2"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "2"}}, + }, + }, + } + + if ok := t.Run("no caches", func(t *testing.T) { + s.cache.SwapIndexCacheWith(noopCache{}) + s.cache.SwapChunksCacheWith(chunkscache.NoopCache{}) + testBucketStore_e2e(t, ctx, s, additionalCases...) + }); !ok { + return + } + + if ok := t.Run("with large, sufficient index cache", func(t *testing.T) { + indexCache, err := indexcache.NewInMemoryIndexCacheWithConfig(s.logger, nil, indexcache.InMemoryIndexCacheConfig{ + MaxItemSize: 1e5, + MaxSize: 2e5, + }) + assert.NoError(t, err) + s.cache.SwapIndexCacheWith(indexCache) + testBucketStore_e2e(t, ctx, s, additionalCases...) + }); !ok { + return + } + + if ok := t.Run("with small index cache", func(t *testing.T) { indexCache2, err := indexcache.NewInMemoryIndexCacheWithConfig(s.logger, nil, indexcache.InMemoryIndexCacheConfig{ MaxItemSize: 50, MaxSize: 100, @@ -502,6 +598,20 @@ func TestBucketStore_e2e(t *testing.T) { assert.NoError(t, err) s.cache.SwapIndexCacheWith(indexCache2) testBucketStore_e2e(t, ctx, s) + }); !ok { + return + } + + t.Run("with large, sufficient index cache, and chunks cache", func(t *testing.T) { + indexCache, err := indexcache.NewInMemoryIndexCacheWithConfig(s.logger, nil, indexcache.InMemoryIndexCacheConfig{ + MaxItemSize: 1e5, + MaxSize: 2e5, + }) + assert.NoError(t, err) + assert.NoError(t, err) + s.cache.SwapIndexCacheWith(indexCache) + s.cache.SwapChunksCacheWith(newInMemoryChunksCache()) + testBucketStore_e2e(t, ctx, s) }) }) } @@ -541,8 +651,8 @@ func TestBucketStore_ManyParts_e2e(t *testing.T) { } func TestBucketStore_Series_ChunksLimiter_e2e(t *testing.T) { - // The query will fetch 2 series from 6 blocks, so we do expect to hit a total of 12 chunks. - expectedChunks := uint64(2 * 6) + // The query will fetch 4 series from 3 blocks each, so we do expect to hit a total of 12 chunks. + expectedChunks := uint64(4 * 3) cases := map[string]struct { maxChunksLimit uint64 @@ -553,6 +663,10 @@ func TestBucketStore_Series_ChunksLimiter_e2e(t *testing.T) { "should succeed if the max chunks limit is not exceeded": { maxChunksLimit: expectedChunks, }, + "should succeed if the max series limit is not exceeded": { + // The streaming case should not count the series twice. + maxSeriesLimit: 4, + }, "should fail if the max chunks limit is exceeded - 422": { maxChunksLimit: expectedChunks - 1, expectedErr: "exceeded chunks limit", @@ -568,36 +682,41 @@ func TestBucketStore_Series_ChunksLimiter_e2e(t *testing.T) { for testName, testData := range cases { t.Run(testName, func(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - bkt := objstore.NewInMemBucket() - - prepConfig := defaultPrepareStoreConfig(t) - prepConfig.chunksLimiterFactory = newStaticChunksLimiterFactory(testData.maxChunksLimit) - prepConfig.seriesLimiterFactory = newStaticSeriesLimiterFactory(testData.maxSeriesLimit) - - s := prepareStoreWithTestBlocks(t, bkt, prepConfig) - assert.NoError(t, s.store.SyncBlocks(ctx)) - - req := &storepb.SeriesRequest{ - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_EQ, Name: "a", Value: "1"}, - }, - MinTime: timestamp.FromTime(minTime), - MaxTime: timestamp.FromTime(maxTime), - } - - srv := newBucketStoreTestServer(t, s.store) - _, _, _, err := srv.Series(context.Background(), req) - - if testData.expectedErr == "" { - assert.NoError(t, err) - } else { - assert.Error(t, err) - assert.True(t, strings.Contains(err.Error(), testData.expectedErr)) - status, ok := status.FromError(err) - assert.Equal(t, true, ok) - assert.Equal(t, testData.expectedCode, status.Code()) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + bkt := objstore.NewInMemBucket() + + prepConfig := defaultPrepareStoreConfig(t) + prepConfig.chunksLimiterFactory = newStaticChunksLimiterFactory(testData.maxChunksLimit) + prepConfig.seriesLimiterFactory = newStaticSeriesLimiterFactory(testData.maxSeriesLimit) + + s := prepareStoreWithTestBlocks(t, bkt, prepConfig) + assert.NoError(t, s.store.SyncBlocks(ctx)) + + req := &storepb.SeriesRequest{ + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_EQ, Name: "a", Value: "1"}, + }, + MinTime: timestamp.FromTime(minTime), + MaxTime: timestamp.FromTime(maxTime), + StreamingChunksBatchSize: uint64(streamingBatchSize), + } + + srv := newBucketStoreTestServer(t, s.store) + _, _, _, err := srv.Series(context.Background(), req) + + if testData.expectedErr == "" { + assert.NoError(t, err) + } else { + assert.Error(t, err) + assert.True(t, strings.Contains(err.Error(), testData.expectedErr)) + status, ok := status.FromError(err) + assert.Equal(t, true, ok) + assert.Equal(t, testData.expectedCode, status.Code()) + } + }) } }) } @@ -819,41 +938,46 @@ func TestBucketStore_LabelValues_e2e(t *testing.T) { } func TestBucketStore_ValueTypes_e2e(t *testing.T) { - foreachStore(t, func(t *testing.T, newSuite suiteFactory) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + foreachStore(t, func(t *testing.T, newSuite suiteFactory) { - s := newSuite() - - mint, maxt := s.store.TimeRange() - assert.Equal(t, s.minTime, mint) - assert.Equal(t, s.maxTime, maxt) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() - req := &storepb.SeriesRequest{ - MinTime: mint, - MaxTime: maxt, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "a", Value: "1|2"}, - }, - SkipChunks: false, - } + s := newSuite() - srv := newBucketStoreTestServer(t, s.store) - seriesSet, _, _, err := srv.Series(ctx, req) - require.NoError(t, err) + mint, maxt := s.store.TimeRange() + assert.Equal(t, s.minTime, mint) + assert.Equal(t, s.maxTime, maxt) - counts := map[storepb.Chunk_Encoding]int{} - for _, series := range seriesSet { - for _, chunk := range series.Chunks { - counts[chunk.Raw.Type]++ - } - } - for _, chunkType := range []storepb.Chunk_Encoding{storepb.Chunk_XOR, storepb.Chunk_Histogram, storepb.Chunk_FloatHistogram} { - count, ok := counts[chunkType] - assert.True(t, ok, fmt.Sprintf("value type %s is not present", storepb.Chunk_Encoding_name[int32(chunkType)])) - assert.NotEmpty(t, count) - } - }) + req := &storepb.SeriesRequest{ + MinTime: mint, + MaxTime: maxt, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "a", Value: "1|2"}, + }, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } + + srv := newBucketStoreTestServer(t, s.store) + seriesSet, _, _, err := srv.Series(ctx, req) + require.NoError(t, err) + + counts := map[storepb.Chunk_Encoding]int{} + for _, series := range seriesSet { + for _, chunk := range series.Chunks { + counts[chunk.Raw.Type]++ + } + } + for _, chunkType := range []storepb.Chunk_Encoding{storepb.Chunk_XOR, storepb.Chunk_Histogram, storepb.Chunk_FloatHistogram} { + count, ok := counts[chunkType] + assert.True(t, ok, fmt.Sprintf("value type %s is not present", storepb.Chunk_Encoding_name[int32(chunkType)])) + assert.NotEmpty(t, count) + } + }) + }) + } } func emptyToNil(values []string) []string { diff --git a/pkg/storegateway/bucket_index_postings.go b/pkg/storegateway/bucket_index_postings.go index 20cc41792f4..f3bd0708ecc 100644 --- a/pkg/storegateway/bucket_index_postings.go +++ b/pkg/storegateway/bucket_index_postings.go @@ -280,7 +280,6 @@ func filterPostingsByCachedShardHash(ps []storage.SeriesRef, shard *sharding.Sha // Shrink the size. ps = ps[:writeIdx] - return ps } diff --git a/pkg/storegateway/bucket_index_reader.go b/pkg/storegateway/bucket_index_reader.go index fbc5b20c4c7..a8cc3daccae 100644 --- a/pkg/storegateway/bucket_index_reader.go +++ b/pkg/storegateway/bucket_index_reader.go @@ -763,12 +763,12 @@ func (l *bucketIndexLoadedSeries) addSeries(ref storage.SeriesRef, data []byte) // Error is returned on decoding error or if the reference does not resolve to a known series. // // It's NOT safe to call this function concurrently with addSeries(). -func (l *bucketIndexLoadedSeries) unsafeLoadSeries(ref storage.SeriesRef, chks *[]chunks.Meta, skipChunks bool, stats *queryStats, lsetPool *pool.SlabPool[symbolizedLabel]) (ok bool, _ []symbolizedLabel, err error) { +func (l *bucketIndexLoadedSeries) unsafeLoadSeries(ref storage.SeriesRef, chks *[]chunks.Meta, strategy seriesIteratorStrategy, stats *queryStats, lsetPool *pool.SlabPool[symbolizedLabel]) (ok bool, _ []symbolizedLabel, err error) { b, ok := l.series[ref] if !ok { return false, nil, errors.Errorf("series %d not found", ref) } stats.seriesProcessed++ stats.seriesProcessedSizeSum += len(b) - return decodeSeries(b, lsetPool, chks, skipChunks) + return decodeSeries(b, lsetPool, chks, strategy) } diff --git a/pkg/storegateway/bucket_store_server_test.go b/pkg/storegateway/bucket_store_server_test.go index 05af7b37972..afd6c6d4cfa 100644 --- a/pkg/storegateway/bucket_store_server_test.go +++ b/pkg/storegateway/bucket_store_server_test.go @@ -95,9 +95,10 @@ func newStoreGatewayTestServer(t testing.TB, store storegatewaypb.StoreGatewaySe // via the gRPC stream. func (s *storeTestServer) Series(ctx context.Context, req *storepb.SeriesRequest) (seriesSet []*storepb.Series, warnings storage.Warnings, hints hintspb.SeriesResponseHints, err error) { var ( - conn *grpc.ClientConn - stream storepb.Store_SeriesClient - res *storepb.SeriesResponse + conn *grpc.ClientConn + stream storepb.Store_SeriesClient + res *storepb.SeriesResponse + streamingSeriesSet []*storepb.StreamingSeries ) // Create a gRPC connection to the server. @@ -146,6 +147,10 @@ func (s *storeTestServer) Series(ctx context.Context, req *storepb.SeriesRequest } if recvSeries := res.GetSeries(); recvSeries != nil { + if !req.SkipChunks && req.StreamingChunksBatchSize > 0 { + err = errors.New("got a normal series when streaming was enabled") + return + } var recvSeriesData []byte // We use a pool for the chunks and may use other pools in the future. @@ -166,6 +171,92 @@ func (s *storeTestServer) Series(ctx context.Context, req *storepb.SeriesRequest seriesSet = append(seriesSet, copiedSeries) } + + if recvSeries := res.GetStreamingSeries(); recvSeries != nil { + if req.StreamingChunksBatchSize == 0 || req.SkipChunks { + err = errors.New("got a streaming series when streaming was disabled") + return + } + + var recvSeriesData []byte + + // We prefer to stay on the safest side at this stage + // so we do a marshal+unmarshal to copy the whole series. + recvSeriesData, err = recvSeries.Marshal() + if err != nil { + err = errors.Wrap(err, "marshal received series") + return + } + + copiedSeries := &storepb.StreamingSeriesBatch{} + if err = copiedSeries.Unmarshal(recvSeriesData); err != nil { + err = errors.Wrap(err, "unmarshal received series") + return + } + + streamingSeriesSet = append(streamingSeriesSet, copiedSeries.Series...) + + if recvSeries.IsEndOfSeriesStream { + break + } + } + } + + if req.StreamingChunksBatchSize > 0 && !req.SkipChunks { + // Get the streaming chunks. + idx := -1 + for idx < len(streamingSeriesSet)-1 { + // We don't expect EOF errors here. + res, err = stream.Recv() + if err != nil { + return + } + + chksBatch := res.GetStreamingChunks() + for _, chks := range chksBatch.Series { + idx++ + if chksBatch == nil { + err = errors.Errorf("expected streaming chunks, got something else") + return + } + if chks.SeriesIndex != uint64(idx) { + err = errors.Errorf("mismatch in series ref when getting streaming chunks, exp %d, got %d", idx, chks.SeriesIndex) + return + } + + // We prefer to stay on the safest side at this stage + // so we do a marshal+unmarshal to copy the whole chunks. + var data []byte + data, err = chks.Marshal() + if err != nil { + err = errors.Wrap(err, "marshal received series") + return + } + + copiedChunks := &storepb.StreamingChunks{} + if err = copiedChunks.Unmarshal(data); err != nil { + err = errors.Wrap(err, "unmarshal received series") + return + } + + seriesSet = append(seriesSet, &storepb.Series{ + Labels: streamingSeriesSet[idx].Labels, + Chunks: copiedChunks.Chunks, + }) + } + } + + res, err = stream.Recv() + for err == nil { + if res.GetHints() == nil && res.GetStats() == nil { + err = errors.Errorf("got unexpected response type") + break + } + res, err = stream.Recv() + } + if errors.Is(err, io.EOF) { + err = nil + } } return diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index d2deba0d1e4..078f562b928 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -1164,7 +1164,7 @@ func loadSeries(ctx context.Context, tb test.TB, postings []storage.SeriesRef, i indexr.block.meta, nil, nil, - true, + noChunkRefs, 0, 0, "", @@ -1409,40 +1409,49 @@ func benchBucketSeries(t test.TB, skipChunk bool, samplesPerSeries, totalSeries ExpectedSeries: series[:seriesCut], }) } - runTestServerSeries(t, st, bCases...) + streamingBatchSizes := []int{0} if !t.IsBenchmark() { - if !skipChunk { - assert.Zero(t, seriesChunksSlicePool.(*pool.TrackedPool).Balance.Load()) - assert.Zero(t, chunksSlicePool.(*pool.TrackedPool).Balance.Load()) + streamingBatchSizes = []int{0, 1, 5} + } + for _, streamingBatchSize := range streamingBatchSizes { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t test.TB) { + runTestServerSeries(t, st, streamingBatchSize, bCases...) - assert.Greater(t, int(seriesChunksSlicePool.(*pool.TrackedPool).Gets.Load()), 0) - assert.Greater(t, int(chunksSlicePool.(*pool.TrackedPool).Gets.Load()), 0) - } + if !t.IsBenchmark() { + if !skipChunk { + assert.Zero(t, seriesChunksSlicePool.(*pool.TrackedPool).Balance.Load()) + assert.Zero(t, chunksSlicePool.(*pool.TrackedPool).Balance.Load()) - for _, b := range st.blocks { - // NOTE(bwplotka): It is 4 x 1.0 for 100mln samples. Kind of make sense: long series. - assert.Equal(t, 0.0, promtest.ToFloat64(b.metrics.seriesRefetches)) - } + assert.Greater(t, int(seriesChunksSlicePool.(*pool.TrackedPool).Gets.Load()), 0) + assert.Greater(t, int(chunksSlicePool.(*pool.TrackedPool).Gets.Load()), 0) + } - // Check exposed metrics. - assertHistograms := map[string]bool{ - "cortex_bucket_store_series_request_stage_duration_seconds": true, - "cortex_bucket_store_series_batch_preloading_load_duration_seconds": st.maxSeriesPerBatch < totalSeries, // Tracked only when a request is split in multiple batches. - "cortex_bucket_store_series_batch_preloading_wait_duration_seconds": st.maxSeriesPerBatch < totalSeries, // Tracked only when a request is split in multiple batches. - "cortex_bucket_store_series_refs_fetch_duration_seconds": true, - } + for _, b := range st.blocks { + // NOTE(bwplotka): It is 4 x 1.0 for 100mln samples. Kind of make sense: long series. + assert.Equal(t, 0.0, promtest.ToFloat64(b.metrics.seriesRefetches)) + } + + // Check exposed metrics. + assertHistograms := map[string]bool{ + "cortex_bucket_store_series_request_stage_duration_seconds": true, + "cortex_bucket_store_series_batch_preloading_load_duration_seconds": st.maxSeriesPerBatch < totalSeries, // Tracked only when a request is split in multiple batches. + "cortex_bucket_store_series_batch_preloading_wait_duration_seconds": st.maxSeriesPerBatch < totalSeries, // Tracked only when a request is split in multiple batches. + "cortex_bucket_store_series_refs_fetch_duration_seconds": true, + } - metrics, err := dskit_metrics.NewMetricFamilyMapFromGatherer(reg) - require.NoError(t, err) + metrics, err := dskit_metrics.NewMetricFamilyMapFromGatherer(reg) + require.NoError(t, err) - for metricName, expected := range assertHistograms { - if count := metrics.SumHistograms(metricName).Count(); expected { - assert.Greater(t, count, uint64(0), "metric name: %s", metricName) - } else { - assert.Equal(t, uint64(0), count, "metric name: %s", metricName) + for metricName, expected := range assertHistograms { + if count := metrics.SumHistograms(metricName).Count(); expected { + assert.Greater(t, count, uint64(0), "metric name: %s", metricName) + } else { + assert.Equal(t, uint64(0), count, "metric name: %s", metricName) + } + } } - } + }) } } @@ -1556,16 +1565,16 @@ func TestBucketStore_Series_Concurrency(t *testing.T) { marshalledHints, err := types.MarshalAny(hints) require.NoError(t, err) - req := &storepb.SeriesRequest{ - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_EQ, Name: labels.MetricName, Value: "test_metric"}, - }, - Hints: marshalledHints, - } - - runRequest := func(t *testing.T, store *BucketStore) { + runRequest := func(t *testing.T, store *BucketStore, streamBatchSize int) { + req := &storepb.SeriesRequest{ + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_EQ, Name: labels.MetricName, Value: "test_metric"}, + }, + Hints: marshalledHints, + StreamingChunksBatchSize: uint64(streamBatchSize), + } srv := newBucketStoreTestServer(t, store) seriesSet, warnings, _, err := srv.Series(context.Background(), req) require.NoError(t, err) @@ -1582,57 +1591,61 @@ func TestBucketStore_Series_Concurrency(t *testing.T) { // Run the test with different batch sizes. for _, batchSize := range []int{len(expectedSeries) / 100, len(expectedSeries) * 2} { t.Run(fmt.Sprintf("batch size: %d", batchSize), func(t *testing.T) { - // Reset the memory pool tracker. - seriesChunkRefsSetPool.(*pool.TrackedPool).Reset() + for _, streamBatchSize := range []int{0, 10} { + t.Run(fmt.Sprintf("streamBatchSize:%d", streamBatchSize), func(t *testing.T) { + // Reset the memory pool tracker. + seriesChunkRefsSetPool.(*pool.TrackedPool).Reset() - metaFetcher, err := block.NewMetaFetcher(logger, 1, instrumentedBucket, "", nil, nil) - assert.NoError(t, err) + metaFetcher, err := block.NewMetaFetcher(logger, 1, instrumentedBucket, "", nil, nil) + assert.NoError(t, err) - // Create the bucket store. - store, err := NewBucketStore( - "test-user", - instrumentedBucket, - metaFetcher, - tmpDir, - batchSize, - 1, - selectAllStrategy{}, - newStaticChunksLimiterFactory(0), - newStaticSeriesLimiterFactory(0), - newGapBasedPartitioners(mimir_tsdb.DefaultPartitionerMaxGapSize, nil), - 1, - mimir_tsdb.DefaultPostingOffsetInMemorySampling, - indexheader.Config{}, - false, // Lazy index-header loading disabled. - 0, - hashcache.NewSeriesHashCache(1024*1024), - NewBucketStoreMetrics(nil), - WithLogger(logger), - ) - require.NoError(t, err) - require.NoError(t, store.SyncBlocks(ctx)) - - // Run workers. - wg := sync.WaitGroup{} - wg.Add(numWorkers) - - for c := 0; c < numWorkers; c++ { - go func() { - defer wg.Done() - - for r := 0; r < numRequestsPerWorker; r++ { - runRequest(t, store) + // Create the bucket store. + store, err := NewBucketStore( + "test-user", + instrumentedBucket, + metaFetcher, + tmpDir, + batchSize, + 1, + selectAllStrategy{}, + newStaticChunksLimiterFactory(0), + newStaticSeriesLimiterFactory(0), + newGapBasedPartitioners(mimir_tsdb.DefaultPartitionerMaxGapSize, nil), + 1, + mimir_tsdb.DefaultPostingOffsetInMemorySampling, + indexheader.Config{}, + false, // Lazy index-header loading disabled. + 0, + hashcache.NewSeriesHashCache(1024*1024), + NewBucketStoreMetrics(nil), + WithLogger(logger), + ) + require.NoError(t, err) + require.NoError(t, store.SyncBlocks(ctx)) + + // Run workers. + wg := sync.WaitGroup{} + wg.Add(numWorkers) + + for c := 0; c < numWorkers; c++ { + go func() { + defer wg.Done() + + for r := 0; r < numRequestsPerWorker; r++ { + runRequest(t, store, streamBatchSize) + } + }() } - }() - } - // Wait until all workers have done. - wg.Wait() + // Wait until all workers have done. + wg.Wait() - // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from - // pool have put back. - assert.Greater(t, seriesChunkRefsSetPool.(*pool.TrackedPool).Gets.Load(), int64(0)) - assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*pool.TrackedPool).Balance.Load()) + // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from + // pool have put back. + assert.Greater(t, seriesChunkRefsSetPool.(*pool.TrackedPool).Gets.Load(), int64(0)) + assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*pool.TrackedPool).Balance.Load()) + }) + } }) } } @@ -1879,7 +1892,11 @@ func TestBucketStore_Series_RequestAndResponseHints(t *testing.T) { tb, store, seriesSet1, seriesSet2, block1, block2, cleanup := setupStoreForHintsTest(t, 5000) tb.Cleanup(cleanup) - runTestServerSeries(tb, store, newTestCases(seriesSet1, seriesSet2, block1, block2)...) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + runTestServerSeries(tb, store, streamingBatchSize, newTestCases(seriesSet1, seriesSet2, block1, block2)...) + }) + } } func TestBucketStore_Series_ErrorUnmarshallingRequestHints(t *testing.T) { @@ -1998,6 +2015,13 @@ func TestBucketStore_Series_CanceledRequest(t *testing.T) { s, ok := status.FromError(err) assert.True(t, ok) assert.Equal(t, codes.Canceled, s.Code()) + + req.StreamingChunksBatchSize = 10 + _, _, _, err = srv.Series(ctx, req) + assert.Error(t, err) + s, ok = status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.Canceled, s.Code()) } func TestBucketStore_Series_InvalidRequest(t *testing.T) { @@ -2189,28 +2213,33 @@ func testBucketStoreSeriesBlockWithMultipleChunks( for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - req := &storepb.SeriesRequest{ - MinTime: testData.reqMinTime, - MaxTime: testData.reqMaxTime, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_EQ, Name: "__name__", Value: "test"}, - }, - } + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + req := &storepb.SeriesRequest{ + MinTime: testData.reqMinTime, + MaxTime: testData.reqMaxTime, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_EQ, Name: "__name__", Value: "test"}, + }, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } - seriesSet, _, _, err := srv.Series(context.Background(), req) - assert.NoError(t, err) - assert.True(t, len(seriesSet) == 1) + seriesSet, _, _, err := srv.Series(context.Background(), req) + assert.NoError(t, err) + assert.True(t, len(seriesSet) == 1) - // Count the number of samples in the returned chunks. - numSamples := 0 - for _, rawChunk := range seriesSet[0].Chunks { - decodedChunk, err := chunkenc.FromData(encoding, rawChunk.Raw.Data) - assert.NoError(t, err) + // Count the number of samples in the returned chunks. + numSamples := 0 + for _, rawChunk := range seriesSet[0].Chunks { + decodedChunk, err := chunkenc.FromData(encoding, rawChunk.Raw.Data) + assert.NoError(t, err) - numSamples += decodedChunk.NumSamples() - } + numSamples += decodedChunk.NumSamples() + } - assert.True(t, testData.expectedSamples == numSamples, "expected: %d, actual: %d", testData.expectedSamples, numSamples) + assert.True(t, testData.expectedSamples == numSamples, "expected: %d, actual: %d", testData.expectedSamples, numSamples) + }) + } }) } } @@ -2312,21 +2341,26 @@ func TestBucketStore_Series_Limits(t *testing.T) { assert.NoError(t, err) assert.NoError(t, store.SyncBlocks(ctx)) - req := &storepb.SeriesRequest{ - MinTime: minTime, - MaxTime: maxTime, - Matchers: testData.reqMatchers, - } - srv := newBucketStoreTestServer(t, store) - seriesSet, _, _, err := srv.Series(ctx, req) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize: %d", streamingBatchSize), func(t *testing.T) { + req := &storepb.SeriesRequest{ + MinTime: minTime, + MaxTime: maxTime, + Matchers: testData.reqMatchers, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } - if testData.expectedErr != "" { - require.Error(t, err) - assert.ErrorContains(t, err, testData.expectedErr) - } else { - require.NoError(t, err) - assert.Len(t, seriesSet, testData.expectedSeries) + seriesSet, _, _, err := srv.Series(ctx, req) + + if testData.expectedErr != "" { + require.Error(t, err) + assert.ErrorContains(t, err, testData.expectedErr) + } else { + require.NoError(t, err) + assert.Len(t, seriesSet, testData.expectedSeries) + } + }) } }) } @@ -2796,11 +2830,12 @@ type seriesCase struct { } // runTestServerSeries runs tests against given cases. -func runTestServerSeries(t test.TB, store *BucketStore, cases ...*seriesCase) { +func runTestServerSeries(t test.TB, store *BucketStore, streamingBatchSize int, cases ...*seriesCase) { for _, c := range cases { t.Run(c.Name, func(t test.TB) { srv := newBucketStoreTestServer(t, store) + c.Req.StreamingChunksBatchSize = uint64(streamingBatchSize) t.ResetTimer() for i := 0; i < t.N(); i++ { seriesSet, warnings, hints, err := srv.Series(context.Background(), c.Req) diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index 1511af6a57d..fcb81838c82 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -1050,34 +1050,38 @@ func TestStoreGateway_SeriesQueryingShouldRemoveExternalLabels(t *testing.T) { srv := newStoreGatewayTestServer(t, g) - // Query back all series. - req := &storepb.SeriesRequest{ - MinTime: minT, - MaxTime: maxT, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, - }, - } - - seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) - require.NoError(t, err) - assert.Empty(t, warnings) - assert.Len(t, seriesSet, numSeries) - - for seriesID := 0; seriesID < numSeries; seriesID++ { - actual := seriesSet[seriesID] - - // Ensure Mimir external labels have been removed. - assert.Equal(t, []mimirpb.LabelAdapter{{Name: "series_id", Value: strconv.Itoa(seriesID)}}, actual.Labels) - - // Ensure samples have been correctly queried. The store-gateway doesn't deduplicate chunks, - // so the same sample is returned twice because in this test we query two identical blocks. - samples, err := readSamplesFromChunks(actual.Chunks) - require.NoError(t, err) - assert.Equal(t, []sample{ - {t: minT + (step * int64(seriesID)), v: float64(seriesID)}, - {t: minT + (step * int64(seriesID)), v: float64(seriesID)}, - }, samples) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + // Query back all series. + req := &storepb.SeriesRequest{ + MinTime: minT, + MaxTime: maxT, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, + }, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } + seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) + require.NoError(t, err) + assert.Empty(t, warnings) + assert.Len(t, seriesSet, numSeries) + + for seriesID := 0; seriesID < numSeries; seriesID++ { + actual := seriesSet[seriesID] + + // Ensure Mimir external labels have been removed. + assert.Equal(t, []mimirpb.LabelAdapter{{Name: "series_id", Value: strconv.Itoa(seriesID)}}, actual.Labels) + + // Ensure samples have been correctly queried. The store-gateway doesn't deduplicate chunks, + // so the same sample is returned twice because in this test we query two identical blocks. + samples, err := readSamplesFromChunks(actual.Chunks) + require.NoError(t, err) + assert.Equal(t, []sample{ + {t: minT + (step * int64(seriesID)), v: float64(seriesID)}, + {t: minT + (step * int64(seriesID)), v: float64(seriesID)}, + }, samples) + } + }) } }) } @@ -1124,58 +1128,63 @@ func TestStoreGateway_Series_QuerySharding(t *testing.T) { }, } - // Prepare the storage dir. - bucketClient, storageDir := mimir_testutil.PrepareFilesystemBucket(t) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + // Prepare the storage dir. + bucketClient, storageDir := mimir_testutil.PrepareFilesystemBucket(t) + + // Generate a TSDB block in the storage dir, containing the fixture series. + mockTSDBWithGenerator(t, path.Join(storageDir, userID), func() func() (bool, labels.Labels, int64, float64) { + nextID := 0 + return func() (bool, labels.Labels, int64, float64) { + if nextID >= len(series) { + return false, labels.Labels{}, 0, 0 + } - // Generate a TSDB block in the storage dir, containing the fixture series. - mockTSDBWithGenerator(t, path.Join(storageDir, userID), func() func() (bool, labels.Labels, int64, float64) { - nextID := 0 - return func() (bool, labels.Labels, int64, float64) { - if nextID >= len(series) { - return false, labels.Labels{}, 0, 0 - } + nextSeries := series[nextID] + nextID++ - nextSeries := series[nextID] - nextID++ + return true, nextSeries, util.TimeToMillis(time.Now().Add(-time.Duration(nextID) * time.Second)), float64(nextID) + } + }()) - return true, nextSeries, util.TimeToMillis(time.Now().Add(-time.Duration(nextID) * time.Second)), float64(nextID) - } - }()) + createBucketIndex(t, bucketClient, userID) - createBucketIndex(t, bucketClient, userID) - - // Create a store-gateway. - gatewayCfg := mockGatewayConfig() - storageCfg := mockStorageConfig(t) - storageCfg.BucketStore.BucketIndex.DeprecatedEnabled = true + // Create a store-gateway. + gatewayCfg := mockGatewayConfig() + storageCfg := mockStorageConfig(t) + storageCfg.BucketStore.BucketIndex.DeprecatedEnabled = true - ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) - t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), log.NewNopLogger(), nil, nil) - require.NoError(t, err) - require.NoError(t, services.StartAndAwaitRunning(ctx, g)) - t.Cleanup(func() { assert.NoError(t, services.StopAndAwaitTerminated(ctx, g)) }) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), log.NewNopLogger(), nil, nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(ctx, g)) + t.Cleanup(func() { assert.NoError(t, services.StopAndAwaitTerminated(ctx, g)) }) - srv := newStoreGatewayTestServer(t, g) + srv := newStoreGatewayTestServer(t, g) - for testName, testData := range tests { - t.Run(testName, func(t *testing.T) { - req := &storepb.SeriesRequest{ - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, - Matchers: testData.matchers, - } + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + req := &storepb.SeriesRequest{ + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + Matchers: testData.matchers, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } - seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) - require.NoError(t, err) - assert.Empty(t, warnings) + seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) + require.NoError(t, err) + assert.Empty(t, warnings) - actualMetrics := make([]string, 0, len(seriesSet)) - for _, s := range seriesSet { - actualMetrics = append(actualMetrics, s.PromLabels().Get(labels.MetricName)) + actualMetrics := make([]string, 0, len(seriesSet)) + for _, s := range seriesSet { + actualMetrics = append(actualMetrics, s.PromLabels().Get(labels.MetricName)) + } + assert.ElementsMatch(t, testData.expectedMetrics, actualMetrics) + }) } - assert.ElementsMatch(t, testData.expectedMetrics, actualMetrics) }) } } @@ -1237,30 +1246,35 @@ func TestStoreGateway_Series_QueryShardingShouldGuaranteeSeriesShardingConsisten // Query all series, 1 shard at a time. for shardID := 0; shardID < numShards; shardID++ { - shardLabel := sharding.FormatShardIDLabelValue(uint64(shardID), numShards) - expectedSeriesIDs := expectedSeriesIDByShard[shardLabel] - - req := &storepb.SeriesRequest{ - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: "series_id", Value: ".+"}, - {Type: storepb.LabelMatcher_EQ, Name: sharding.ShardLabel, Value: shardLabel}, - }, - } + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + shardLabel := sharding.FormatShardIDLabelValue(uint64(shardID), numShards) + expectedSeriesIDs := expectedSeriesIDByShard[shardLabel] + + req := &storepb.SeriesRequest{ + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "series_id", Value: ".+"}, + {Type: storepb.LabelMatcher_EQ, Name: sharding.ShardLabel, Value: shardLabel}, + }, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } - seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) - require.NoError(t, err) - assert.Empty(t, warnings) - require.Greater(t, len(seriesSet), 0) + seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) + require.NoError(t, err) + assert.Empty(t, warnings) + require.Greater(t, len(seriesSet), 0) - for _, series := range seriesSet { - // Ensure the series below to the right shard. - seriesLabels := mimirpb.FromLabelAdaptersToLabels(series.Labels) - seriesID, err := strconv.Atoi(seriesLabels.Get("series_id")) - require.NoError(t, err) + for _, series := range seriesSet { + // Ensure the series below to the right shard. + seriesLabels := mimirpb.FromLabelAdaptersToLabels(series.Labels) + seriesID, err := strconv.Atoi(seriesLabels.Get("series_id")) + require.NoError(t, err) - assert.Contains(t, expectedSeriesIDs, seriesID, "series:", seriesLabels.String()) + assert.Contains(t, expectedSeriesIDs, seriesID, "series:", seriesLabels.String()) + } + }) } } } @@ -1312,63 +1326,69 @@ func TestStoreGateway_Series_QueryShardingConcurrency(t *testing.T) { srv := newStoreGatewayTestServer(t, g) - // Keep track of all responses received (by shard). - responsesMx := sync.Mutex{} - responses := make(map[int][][]*storepb.Series) - - wg := sync.WaitGroup{} - wg.Add(numQueries) - - for i := 0; i < numQueries; i++ { - go func(shardIndex int) { - defer wg.Done() - - req := &storepb.SeriesRequest{ - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, - Matchers: []storepb.LabelMatcher{ - {Type: storepb.LabelMatcher_RE, Name: labels.MetricName, Value: ".*"}, - {Type: storepb.LabelMatcher_EQ, Name: sharding.ShardLabel, Value: sharding.ShardSelector{ - ShardIndex: uint64(shardIndex), - ShardCount: uint64(shardCount), - }.LabelValue()}, - }, - } + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + // Keep track of all responses received (by shard). + responsesMx := sync.Mutex{} + responses := make(map[int][][]*storepb.Series) + + wg := sync.WaitGroup{} + wg.Add(numQueries) + + for i := 0; i < numQueries; i++ { + go func(shardIndex int) { + defer wg.Done() + + req := &storepb.SeriesRequest{ + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: labels.MetricName, Value: ".*"}, + {Type: storepb.LabelMatcher_EQ, Name: sharding.ShardLabel, Value: sharding.ShardSelector{ + ShardIndex: uint64(shardIndex), + ShardCount: uint64(shardCount), + }.LabelValue()}, + }, + StreamingChunksBatchSize: uint64(streamingBatchSize), + } - seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) - require.NoError(t, err) - assert.Empty(t, warnings) + seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) + require.NoError(t, err) + assert.Empty(t, warnings) - responsesMx.Lock() - responses[shardIndex] = append(responses[shardIndex], seriesSet) - responsesMx.Unlock() - }(i % shardCount) - } + responsesMx.Lock() + responses[shardIndex] = append(responses[shardIndex], seriesSet) + responsesMx.Unlock() + }(i % shardCount) + } + + // Wait until all requests completed. + wg.Wait() - // Wait until all requests completed. - wg.Wait() + // We expect all responses for a given shard contain the same series + // and all shards merged together contain all the series in the TSDB block. + totalSeries := 0 - // We expect all responses for a given shard contain the same series - // and all shards merged together contain all the series in the TSDB block. - totalSeries := 0 + for shardIndex := 0; shardIndex < shardCount; shardIndex++ { + var expected []*storepb.Series - for shardIndex := 0; shardIndex < shardCount; shardIndex++ { - var expected []*storepb.Series + for resIdx, res := range responses[shardIndex] { + // We consider the 1st response for a shard as the expected one + // (all in all we expect all responses to be the same). + if resIdx == 0 { + expected = res + totalSeries += len(res) + continue + } - for resIdx, res := range responses[shardIndex] { - // We consider the 1st response for a shard as the expected one - // (all in all we expect all responses to be the same). - if resIdx == 0 { - expected = res - totalSeries += len(res) - continue + assert.Equalf(t, expected, res, "shard: %d", shardIndex) + } } - assert.Equalf(t, expected, res, "shard: %d", shardIndex) - } + assert.Equal(t, numSeries, totalSeries) + }) } - assert.Equal(t, numSeries, totalSeries) } func TestStoreGateway_SeriesQueryingShouldEnforceMaxChunksPerQueryLimit(t *testing.T) { @@ -1419,44 +1439,49 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxChunksPerQueryLimit(t *testi }, } - for testName, testData := range tests { - t.Run(testName, func(t *testing.T) { - // Customise the limits. - limits := defaultLimitsConfig() - limits.MaxChunksPerQuery = testData.limit - overrides, err := validation.NewOverrides(limits, nil) - require.NoError(t, err) - - // Create a store-gateway used to query back the series from the blocks. - gatewayCfg := mockGatewayConfig() - storageCfg := mockStorageConfig(t) + for _, streamingBatchSize := range []int{0, 1, 5} { + t.Run(fmt.Sprintf("streamingBatchSize=%d", streamingBatchSize), func(t *testing.T) { + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + // Customise the limits. + limits := defaultLimitsConfig() + limits.MaxChunksPerQuery = testData.limit + overrides, err := validation.NewOverrides(limits, nil) + require.NoError(t, err) - ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) - t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + // Create a store-gateway used to query back the series from the blocks. + gatewayCfg := mockGatewayConfig() + storageCfg := mockStorageConfig(t) - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, logger, nil, nil) - require.NoError(t, err) - require.NoError(t, services.StartAndAwaitRunning(ctx, g)) - t.Cleanup(func() { assert.NoError(t, services.StopAndAwaitTerminated(ctx, g)) }) + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) - srv := newStoreGatewayTestServer(t, g) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, logger, nil, nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(ctx, g)) + t.Cleanup(func() { assert.NoError(t, services.StopAndAwaitTerminated(ctx, g)) }) - // Query back all the series (1 chunk per series in this test). - seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) - - if testData.expectedErr != nil { - require.Error(t, err) - assert.IsType(t, testData.expectedErr, err) - s1, ok := status.FromError(errors.Cause(err)) - assert.True(t, ok) - s2, ok := status.FromError(errors.Cause(testData.expectedErr)) - assert.True(t, ok) - assert.True(t, strings.Contains(s1.Message(), s2.Message())) - assert.Equal(t, s1.Code(), s2.Code()) - } else { - require.NoError(t, err) - assert.Empty(t, warnings) - assert.Len(t, seriesSet, chunksQueried) + srv := newStoreGatewayTestServer(t, g) + + // Query back all the series (1 chunk per series in this test). + req.StreamingChunksBatchSize = uint64(streamingBatchSize) + seriesSet, warnings, _, err := srv.Series(setUserIDToGRPCContext(ctx, userID), req) + + if testData.expectedErr != nil { + require.Error(t, err) + assert.IsType(t, testData.expectedErr, err) + s1, ok := status.FromError(errors.Cause(err)) + assert.True(t, ok) + s2, ok := status.FromError(errors.Cause(testData.expectedErr)) + assert.True(t, ok) + assert.True(t, strings.Contains(s1.Message(), s2.Message())) + assert.Equal(t, s1.Code(), s2.Code()) + } else { + require.NoError(t, err) + assert.Empty(t, warnings) + assert.Len(t, seriesSet, chunksQueried) + } + }) } }) } diff --git a/pkg/storegateway/series_chunks.go b/pkg/storegateway/series_chunks.go index 8599c90f497..8ab9b17f53e 100644 --- a/pkg/storegateway/series_chunks.go +++ b/pkg/storegateway/series_chunks.go @@ -183,7 +183,7 @@ func newSeriesChunksSeriesSet(from seriesChunksSetIterator) storepb.SeriesSet { } } -func newSeriesSetWithChunks( +func newChunksPreloadingIterator( ctx context.Context, logger log.Logger, userID string, @@ -193,11 +193,11 @@ func newSeriesSetWithChunks( refsIteratorBatchSize int, stats *safeQueryStats, minT, maxT int64, -) storepb.SeriesSet { +) seriesChunksSetIterator { var iterator seriesChunksSetIterator iterator = newLoadingSeriesChunksSetIterator(ctx, logger, userID, cache, chunkReaders, refsIterator, refsIteratorBatchSize, stats, minT, maxT) iterator = newPreloadingAndStatsTrackingSetIterator[seriesChunksSet](ctx, 1, iterator, stats) - return newSeriesChunksSeriesSet(iterator) + return iterator } // Next advances to the next item. Once the underlying seriesChunksSet has been fully consumed @@ -307,6 +307,7 @@ func (p *preloadingSetIterator[Set]) Err() error { func newPreloadingAndStatsTrackingSetIterator[Set any](ctx context.Context, preloadedSetsCount int, iterator genericIterator[Set], stats *safeQueryStats) genericIterator[Set] { // Track the time spent loading batches (including preloading). + numBatches := 0 iterator = newNextDurationMeasuringIterator[Set](iterator, func(duration time.Duration, hasNext bool) { stats.update(func(stats *queryStats) { stats.streamingSeriesBatchLoadDuration += duration @@ -314,8 +315,9 @@ func newPreloadingAndStatsTrackingSetIterator[Set any](ctx context.Context, prel // This function is called for each Next() invocation, so we can use it to measure // into how many batches the request has been split. if hasNext { - stats.streamingSeriesBatchCount++ + numBatches++ } + stats.streamingSeriesBatchCount = numBatches }) }) @@ -419,7 +421,6 @@ func (c *loadingSeriesChunksSetIterator) Next() (retHasNext bool) { c.recordCachedChunks(cachedRanges) } c.chunkReaders.reset() - for sIdx, s := range nextUnloaded.series { nextSet.series[sIdx].lset = s.lset nextSet.series[sIdx].chks = nextSet.newSeriesAggrChunkSlice(s.numChunks()) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index ce5587bc78e..f1459a67ab3 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -719,7 +719,7 @@ type loadingSeriesChunkRefsSetIterator struct { blockID ulid.ULID shard *sharding.ShardSelector seriesHasher seriesHasher - skipChunks bool + strategy seriesIteratorStrategy minTime, maxTime int64 tenantID string chunkRangesPerSeries int @@ -741,19 +741,36 @@ func openBlockSeriesChunkRefsSetsIterator( matchers []*labels.Matcher, // Series matchers. shard *sharding.ShardSelector, // Shard selector. seriesHasher seriesHasher, - skipChunks bool, // If true chunks are not loaded and minTime/maxTime are ignored. + strategy seriesIteratorStrategy, minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true). chunkRangesPerSeries int, stats *safeQueryStats, + reuse *reusedPostingsAndMatchers, // If this is not nil, these posting and matchers are used as it is without fetching new ones. logger log.Logger, ) (seriesChunkRefsSetIterator, error) { if batchSize <= 0 { return nil, errors.New("set size must be a positive number") } - ps, pendingMatchers, err := indexr.ExpandedPostings(ctx, matchers, stats) - if err != nil { - return nil, errors.Wrap(err, "expanded matching postings") + var ( + ps []storage.SeriesRef + pendingMatchers []*labels.Matcher + fetchPostings = true + ) + if reuse != nil { + fetchPostings = !reuse.isSet() + ps = reuse.ps + pendingMatchers = reuse.matchers + } + if fetchPostings { + var err error + ps, pendingMatchers, err = indexr.ExpandedPostings(ctx, matchers, stats) + if err != nil { + return nil, errors.Wrap(err, "expanded matching postings") + } + if reuse != nil { + reuse.set(ps, pendingMatchers) + } } var iterator seriesChunkRefsSetIterator @@ -766,7 +783,7 @@ func openBlockSeriesChunkRefsSetsIterator( blockMeta, shard, seriesHasher, - skipChunks, + strategy, minTime, maxTime, tenantID, @@ -780,6 +797,31 @@ func openBlockSeriesChunkRefsSetsIterator( return seriesStreamingFetchRefsDurationIterator(iterator, stats), nil } +// reusedPostings is used to share the postings and matches across function calls for re-use +// in case of streaming series. We have it as a separate struct so that we can give a safe way +// to use it by making a copy where required. You can use it to put items only once. +type reusedPostingsAndMatchers struct { + ps []storage.SeriesRef + matchers []*labels.Matcher + filled bool +} + +func (p *reusedPostingsAndMatchers) set(ps []storage.SeriesRef, matchers []*labels.Matcher) { + if p.filled { + // We already have something here. + return + } + // Postings list can be modified later, so we make a copy here. + p.ps = make([]storage.SeriesRef, len(ps)) + copy(p.ps, ps) + p.matchers = matchers + p.filled = true +} + +func (p *reusedPostingsAndMatchers) isSet() bool { + return p.filled +} + // seriesStreamingFetchRefsDurationIterator tracks the time spent loading series and chunk refs. func seriesStreamingFetchRefsDurationIterator(iterator seriesChunkRefsSetIterator, stats *safeQueryStats) genericIterator[seriesChunkRefsSet] { return newNextDurationMeasuringIterator[seriesChunkRefsSet](iterator, func(duration time.Duration, _ bool) { @@ -789,6 +831,39 @@ func seriesStreamingFetchRefsDurationIterator(iterator seriesChunkRefsSetIterato }) } +// seriesIteratorStrategy defines the strategy to use when loading the series and their chunk refs. +// See below for available options. +type seriesIteratorStrategy byte + +const ( + // By default, the strategy is to fetch series labels AND chunk refs + // for time ranges overlapping mint and maxt provided. + // To change the default behavior, use the flags below this. + defaultStrategy seriesIteratorStrategy = 0 + + // noChunkRefs flag when used by itself fetches only series labels for series in the entire block. + noChunkRefs seriesIteratorStrategy = 0b00000001 + // overlapMintMaxt flag is used together with noChunkRefs. With this, only the series whose + // chunks overlap with [mint, maxt] are selected. + overlapMintMaxt seriesIteratorStrategy = 0b00000010 +) + +func (s seriesIteratorStrategy) isNoChunkRefs() bool { + return s&noChunkRefs != 0 +} + +func (s seriesIteratorStrategy) isOverlapMintMaxt() bool { + return s&overlapMintMaxt != 0 +} + +func (s seriesIteratorStrategy) isNoChunkRefsOnEntireBlock() bool { + return s.isNoChunkRefs() && !s.isOverlapMintMaxt() +} + +func (s seriesIteratorStrategy) isNoChunkRefsAndOverlapMintMaxt() bool { + return s.isNoChunkRefs() && s.isOverlapMintMaxt() +} + func newLoadingSeriesChunkRefsSetIterator( ctx context.Context, postingsSetIterator *postingsSetsIterator, @@ -798,14 +873,14 @@ func newLoadingSeriesChunkRefsSetIterator( blockMeta *block.Meta, shard *sharding.ShardSelector, seriesHasher seriesHasher, - skipChunks bool, + strategy seriesIteratorStrategy, minTime int64, maxTime int64, tenantID string, chunkRangesPerSeries int, logger log.Logger, ) *loadingSeriesChunkRefsSetIterator { - if skipChunks { + if strategy.isNoChunkRefsOnEntireBlock() { minTime, maxTime = blockMeta.MinTime, blockMeta.MaxTime } @@ -818,7 +893,7 @@ func newLoadingSeriesChunkRefsSetIterator( blockID: blockMeta.ULID, shard: shard, seriesHasher: seriesHasher, - skipChunks: skipChunks, + strategy: strategy, minTime: minTime, maxTime: maxTime, tenantID: tenantID, @@ -849,7 +924,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { nextPostings := s.postingsSetIterator.At() var cachedSeriesID cachedSeriesForPostingsID - if s.skipChunks { + if s.strategy.isNoChunkRefsOnEntireBlock() { var err error // Calculate the cache ID before we filter out anything from the postings, // so that the key doesn't depend on the series hash cache or any other filtering we do on the postings list. @@ -899,7 +974,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { } s.currentSet = nextSet - if s.skipChunks && cachedSeriesID.isSet() { + if cachedSeriesID.isSet() { storeCachedSeriesForPostings(s.ctx, s.indexCache, s.tenantID, s.blockID, s.shard, cachedSeriesID, nextSet, s.logger) } return true @@ -927,7 +1002,20 @@ func (s *loadingSeriesChunkRefsSetIterator) symbolizedSet(ctx context.Context, p if err != nil { return symbolizedSeriesChunkRefsSet{}, errors.Wrap(err, "read series") } - if !s.skipChunks { + + switch { + case s.strategy.isNoChunkRefsAndOverlapMintMaxt(): + overlaps := false + for _, m := range metas { + if m.MaxTime >= s.minTime && m.MinTime <= s.maxTime { + overlaps = true + break + } + } + if !overlaps { + series.lset = nil // setting the labels to nil ends up skipping the series + } + case !s.strategy.isNoChunkRefs(): clampLastChunkLength(symbolizedSet.series, metas) series.chunksRanges = metasToRanges(partitionChunks(metas, s.chunkRangesPerSeries, minChunksPerRange), s.blockID, s.minTime, s.maxTime) } @@ -979,15 +1067,22 @@ func clampLastChunkLength(series []symbolizedSeriesChunkRefs, nextSeriesChunkMet // filterSeries filters out series that don't belong to this shard (if sharding is configured) or that don't have any // chunk ranges and skipChunks=false. Empty chunks ranges indicates that the series doesn't have any chunk ranges in the -// requested time range. +// requested time range. filterSeries expects that the number of series matches the number of postings. func (s *loadingSeriesChunkRefsSetIterator) filterSeries(set seriesChunkRefsSet, postings []storage.SeriesRef, stats *queryStats) seriesChunkRefsSet { writeIdx := 0 for sIdx, series := range set.series { - // An empty label set means the series had no chunks in this block, so we skip it. - // No chunk ranges means the series doesn't have a single chunk range in the requested range. - if series.lset.IsEmpty() || (!s.skipChunks && len(series.chunksRanges) == 0) { + + // We skip this series under three conditions: + // 1. The series doesn't have any chunks in this block OR the series didn't have any chunks in the requested time range, + // but also the request didn't require the chunks (i.e. s.strategy.isNoChunkRefs()). This is signified by an empty label set. + if series.lset.IsEmpty() { + continue + } + // 2. The series doesn't have any chunks in the requested time range but the request required the chunks (i.e. !s.strategy.isNoChunkRefs()). + if !s.strategy.isNoChunkRefs() && len(series.chunksRanges) == 0 { continue } + // 3. The series doesn't belong to this shard. if !shardOwned(s.shard, s.seriesHasher, postings[sIdx], series.lset, stats) { continue } @@ -1120,7 +1215,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Err() error { // loadSeries returns a for chunks. It is not safe to use the returned []chunks.Meta after calling loadSeries again func (s *loadingSeriesChunkRefsSetIterator) loadSeries(ref storage.SeriesRef, loadedSeries *bucketIndexLoadedSeries, stats *queryStats, lsetPool *pool.SlabPool[symbolizedLabel]) ([]symbolizedLabel, []chunks.Meta, error) { - ok, lbls, err := loadedSeries.unsafeLoadSeries(ref, &s.chunkMetasBuffer, s.skipChunks, stats, lsetPool) + ok, lbls, err := loadedSeries.unsafeLoadSeries(ref, &s.chunkMetasBuffer, s.strategy, stats, lsetPool) if !ok || err != nil { return nil, nil, errors.Wrap(err, "loadSeries") } diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 7ef213d0a81..d5ed99f1307 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1083,8 +1083,8 @@ func TestLimitingSeriesChunkRefsSetIterator(t *testing.T) { t.Run(testName, func(t *testing.T) { iterator := newLimitingSeriesChunkRefsSetIterator( newSliceSeriesChunkRefsSetIterator(testCase.upstreamErr, testCase.sets...), - &limiter{limit: testCase.chunksLimit}, - &limiter{limit: testCase.seriesLimit}, + &staticLimiter{limit: testCase.chunksLimit}, + &staticLimiter{limit: testCase.seriesLimit}, ) sets := readAllSeriesChunkRefsSet(iterator) @@ -1116,17 +1116,22 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { assert.NoError(t, appender.Commit()) }) - testCases := map[string]struct { + type testCase struct { blockFactory func() *bucketBlock // if nil, defaultTestBlockFactory is used shard *sharding.ShardSelector matchers []*labels.Matcher seriesHasher seriesHasher - skipChunks bool + strategy seriesIteratorStrategy minT, maxT int64 batchSize int expectedSets []seriesChunkRefsSet - }{ + } + + sharedSeriesHasher := cachedSeriesHasher{hashcache.NewSeriesHashCache(1000).GetBlockCache("")} + sharedSeriesHasher2 := cachedSeriesHasher{hashcache.NewSeriesHashCache(1000).GetBlockCache("")} + + testCases := map[string]testCase{ "loads one batch": { minT: 0, maxT: 10000, @@ -1175,11 +1180,11 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { }, }, "skips chunks": { - skipChunks: true, - minT: 0, - maxT: 40, - batchSize: 100, - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, + strategy: noChunkRefs, + minT: 0, + maxT: 40, + batchSize: 100, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, expectedSets: []seriesChunkRefsSet{ {series: []seriesChunkRefs{ {lset: labels.FromStrings("l1", "v1")}, @@ -1249,11 +1254,11 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { }, }, "ignores mixT/maxT when skipping chunks": { - minT: 0, - maxT: 10, - skipChunks: true, - batchSize: 4, - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, + minT: 0, + maxT: 10, + strategy: noChunkRefs, + batchSize: 4, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, expectedSets: []seriesChunkRefsSet{ {series: []seriesChunkRefs{ {lset: labels.FromStrings("l1", "v1")}, @@ -1267,7 +1272,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { blockFactory: largerTestBlockFactory, minT: 0, maxT: math.MaxInt64, - skipChunks: true, // There is still no easy way to assert on the refs of 100K chunks, so we skip them. + strategy: noChunkRefs, // There is still no easy way to assert on the refs of 100K chunks, so we skip them. batchSize: largerTestBlockSeriesCount, matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", ".*")}, expectedSets: func() []seriesChunkRefsSet { @@ -1286,7 +1291,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { blockFactory: largerTestBlockFactory, minT: 0, maxT: math.MaxInt64, - skipChunks: true, // There is still no easy way to assert on the refs of 100K chunks, so we skip them. + strategy: noChunkRefs, // There is still no easy way to assert on the refs of 100K chunks, so we skip them. batchSize: 5000, matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", ".*")}, expectedSets: func() []seriesChunkRefsSet { @@ -1307,27 +1312,139 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { return sets }(), }, + "skip chunks with streaming on block 1": { + minT: 0, + maxT: 25, + batchSize: 100, + strategy: noChunkRefs | overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v1")}, + {lset: labels.FromStrings("l1", "v2")}, + }}, + }, + }, + "skip chunks with streaming on block 2": { + minT: 15, + maxT: 35, + batchSize: 100, + strategy: noChunkRefs | overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-4]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v2")}, + {lset: labels.FromStrings("l1", "v3")}, + }}, + }, + }, + + // If the first test case stored incorrect hashes in the cache, the second test case would fail. + "doesn't pollute the series hash cache with incorrect hashes (pt. 1)": { + minT: 15, + maxT: 45, + seriesHasher: sharedSeriesHasher, + shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, + batchSize: 100, + strategy: noChunkRefs | overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-5]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v2")}, + {lset: labels.FromStrings("l1", "v3")}, + }}, + }, + }, + "doesn't pollute the series hash cache with incorrect hashes (pt. 2)": { + minT: 15, + maxT: 45, + seriesHasher: sharedSeriesHasher, + shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, + batchSize: 100, + strategy: noChunkRefs | overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-5]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v2")}, + {lset: labels.FromStrings("l1", "v3")}, + }}, + }, + }, + "doesn't pollute the series hash cache with incorrect hashes (without streaming; pt. 1)": { + minT: 15, + maxT: 45, + seriesHasher: sharedSeriesHasher2, + shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, + batchSize: 100, + strategy: overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-5]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v2"), chunksRanges: []seriesChunkRefsRange{{refs: []seriesChunkRef{ + {minTime: 20, maxTime: 20, segFileOffset: 234, length: 208}, + }}}}, + {lset: labels.FromStrings("l1", "v3"), chunksRanges: []seriesChunkRefsRange{{refs: []seriesChunkRef{ + {minTime: 30, maxTime: 30, segFileOffset: 442, length: 208}, + }}}}, + }}, + }, + }, + "doesn't pollute the series hash cache with incorrect hashes (without streaming; pt. 2)": { + minT: 15, + maxT: 45, + seriesHasher: sharedSeriesHasher2, + shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, + batchSize: 100, + strategy: overlapMintMaxt, + matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", "v[1-5]")}, + expectedSets: []seriesChunkRefsSet{ + {series: []seriesChunkRefs{ + {lset: labels.FromStrings("l1", "v2"), chunksRanges: []seriesChunkRefsRange{{refs: []seriesChunkRef{ + {minTime: 20, maxTime: 20, segFileOffset: 234, length: 208}, + }}}}, + {lset: labels.FromStrings("l1", "v3"), chunksRanges: []seriesChunkRefsRange{{refs: []seriesChunkRef{ + // The chunk length here is different from the one in pt. 1. + // l1=v4 is outside this shard, so it was excluded, and we couldn't use its chunk refs to infer length. + // The next series is l1=v5, and we still don't know its shard, so its chunks refs are fetched, + // and we can use them to infer the chunk sizes of l1=v3 + {minTime: 30, maxTime: 30, segFileOffset: 442, length: 416}, + }}}}, + }}, + }, + }, } - for testName, testCase := range testCases { - testName, testCase := testName, testCase - t.Run(testName, func(t *testing.T) { - t.Parallel() + sortedTestCases := make([]struct { + name string + tc testCase + }, 0, len(testCases)) + for name, tc := range testCases { + sortedTestCases = append(sortedTestCases, struct { + name string + tc testCase + }{name, tc}) + } + sort.Slice(sortedTestCases, func(i, j int) bool { + return sortedTestCases[i].name < sortedTestCases[j].name + }) + for _, testCase := range sortedTestCases { + testName, tc := testCase.name, testCase.tc + t.Run(testName, func(t *testing.T) { // Setup blockFactory := defaultTestBlockFactory - if testCase.blockFactory != nil { - blockFactory = testCase.blockFactory + if tc.blockFactory != nil { + blockFactory = tc.blockFactory } block := blockFactory() indexr := block.indexReader(selectAllStrategy{}) - postings, _, err := indexr.ExpandedPostings(context.Background(), testCase.matchers, newSafeQueryStats()) + postings, _, err := indexr.ExpandedPostings(context.Background(), tc.matchers, newSafeQueryStats()) require.NoError(t, err) postingsIterator := newPostingsSetsIterator( postings, - testCase.batchSize, + tc.batchSize, ) - hasher := testCase.seriesHasher + hasher := tc.seriesHasher if hasher == nil { hasher = cachedSeriesHasher{hashcache.NewSeriesHashCache(100).GetBlockCache("")} } @@ -1338,11 +1455,11 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { noopCache{}, newSafeQueryStats(), block.meta, - testCase.shard, + tc.shard, hasher, - testCase.skipChunks, - testCase.minT, - testCase.maxT, + tc.strategy, + tc.minT, + tc.maxT, "t1", 1, log.NewNopLogger(), @@ -1351,7 +1468,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { // Tests sets := readAllSeriesChunkRefsSet(loadingIterator) assert.NoError(t, loadingIterator.Err()) - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, testCase.expectedSets, sets) + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, tc.expectedSets, sets) }) } } @@ -1662,6 +1779,10 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { maxT = testCase.maxT } + var strategy seriesIteratorStrategy + if testCase.skipChunks { + strategy |= noChunkRefs + } iterator, err := openBlockSeriesChunkRefsSetsIterator( ctx, testCase.batchSize, @@ -1672,11 +1793,12 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { []*labels.Matcher{testCase.matcher}, nil, cachedSeriesHasher{hashCache}, - testCase.skipChunks, + strategy, minT, maxT, 2, newSafeQueryStats(), + nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -1773,11 +1895,12 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { testCase.matchers, nil, cachedSeriesHasher{hashCache}, - true, // skip chunks since we are testing labels filtering + noChunkRefs, // skip chunks since we are testing labels filtering block.meta.MinTime, block.meta.MaxTime, 2, newSafeQueryStats(), + nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -1824,6 +1947,7 @@ func BenchmarkOpenBlockSeriesChunkRefsSetsIterator(b *testing.B) { hashCache := hashcache.NewSeriesHashCache(1024 * 1024).GetBlockCache(block.meta.ULID.String()) b.ResetTimer() + b.ReportAllocs() for i := 0; i < b.N; i++ { iterator, err := openBlockSeriesChunkRefsSetsIterator( @@ -1836,11 +1960,12 @@ func BenchmarkOpenBlockSeriesChunkRefsSetsIterator(b *testing.B) { testCase.matchers, nil, cachedSeriesHasher{hashCache}, - false, // we don't skip chunks, so we can measure impact in loading chunk refs too + defaultStrategy, // we don't skip chunks, so we can measure impact in loading chunk refs too block.meta.MinTime, block.meta.MaxTime, 2, newSafeQueryStats(), + nil, log.NewNopLogger(), ) require.NoError(b, err) @@ -2384,11 +2509,12 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - true, + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, 1, statsColdCache, + nil, log.NewNopLogger(), ) @@ -2415,11 +2541,12 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - true, + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, 1, statsWarnCache, + nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -2501,6 +2628,46 @@ func TestPostingsSetsIterator(t *testing.T) { } } +func TestReusedPostingsAndMatchers(t *testing.T) { + postingsList := [][]storage.SeriesRef{ + nil, + {}, + {1, 2, 3}, + } + matchersList := [][]*labels.Matcher{ + nil, + {}, + {labels.MustNewMatcher(labels.MatchEqual, "a", "b")}, + } + + for _, firstPostings := range postingsList { + for _, firstMatchers := range matchersList { + for _, secondPostings := range postingsList { + for _, secondMatchers := range matchersList { + r := reusedPostingsAndMatchers{} + require.False(t, r.isSet()) + + verify := func() { + r.set(firstPostings, firstMatchers) + require.True(t, r.isSet()) + if firstPostings == nil { + require.Equal(t, []storage.SeriesRef{}, r.ps) + } else { + require.Equal(t, firstPostings, r.ps) + } + require.Equal(t, firstMatchers, r.matchers) + } + verify() + + // This should not overwrite the first set. + r.set(secondPostings, secondMatchers) + verify() + } + } + } + } +} + type mockSeriesHasher struct { cached map[storage.SeriesRef]uint64 hashes map[string]uint64 @@ -2550,12 +2717,12 @@ func (s *sliceSeriesChunkRefsSetIterator) Err() error { return nil } -type limiter struct { +type staticLimiter struct { limit int current atomic.Uint64 } -func (l *limiter) Reserve(num uint64) error { +func (l *staticLimiter) Reserve(num uint64) error { if l.current.Add(num) > uint64(l.limit) { return errors.New("test limit exceeded") } diff --git a/pkg/storegateway/stats.go b/pkg/storegateway/stats.go index 9cae3e545cf..3d56211950a 100644 --- a/pkg/storegateway/stats.go +++ b/pkg/storegateway/stats.go @@ -174,3 +174,11 @@ func (s *safeQueryStats) export() *queryStats { copied := *s.unsafeStats return &copied } + +// seriesAndChunksCount return the value of mergedSeriesCount and mergedChunksCount fields. +func (s *safeQueryStats) seriesAndChunksCount() (seriesCount, chunksCount int) { + s.unsafeStatsMx.Lock() + defer s.unsafeStatsMx.Unlock() + + return s.unsafeStats.mergedSeriesCount, s.unsafeStats.mergedChunksCount +} diff --git a/pkg/storegateway/storepb/custom.go b/pkg/storegateway/storepb/custom.go index bf3eb29f2ab..3c8653e6bd3 100644 --- a/pkg/storegateway/storepb/custom.go +++ b/pkg/storegateway/storepb/custom.go @@ -39,6 +39,22 @@ func NewStatsResponse(indexBytesFetched int) *SeriesResponse { } } +func NewStreamingSeriesResponse(series *StreamingSeriesBatch) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_StreamingSeries{ + StreamingSeries: series, + }, + } +} + +func NewStreamingChunksResponse(series *StreamingChunksBatch) *SeriesResponse { + return &SeriesResponse{ + Result: &SeriesResponse_StreamingChunks{ + StreamingChunks: series, + }, + } +} + type emptySeriesSet struct{} func (emptySeriesSet) Next() bool { return false } diff --git a/pkg/storegateway/storepb/rpc.pb.go b/pkg/storegateway/storepb/rpc.pb.go index 2c6c900684a..fcfdd75884e 100644 --- a/pkg/storegateway/storepb/rpc.pb.go +++ b/pkg/storegateway/storepb/rpc.pb.go @@ -40,6 +40,18 @@ type SeriesRequest struct { // The content of this field and whether it's supported depends on the // implementation of a specific store. Hints *types.Any `protobuf:"bytes,9,opt,name=hints,proto3" json:"hints,omitempty"` + // If streaming_chunks_batch_size=0, the response must only contain one 'series' at a time + // with the series labels and chunks data sent together. + // If streaming_chunks_batch_size > 0 + // - The store may choose to send the streaming_series/streaming_chunks OR behave as + // if streaming_chunks_batch_size=0 if it does not support streaming series. + // - The store must not send a mix of 'series' and streaming_series/streaming_chunks for a single request. + // - If the store chooses to send streaming series, all the streaming_series must be sent before + // sending any streaming_chunks, with the last streaming_series response containing is_end_of_series_stream=true. + // The order of series in both streaming_series/streaming_chunks must match and the size of the batch must not + // cross streaming_chunks_batch_size, although it can be lower than that. + // The proto field ID is 100 so that we have an option to bring back compatibility with Thanos' storage API. + StreamingChunksBatchSize uint64 `protobuf:"varint,100,opt,name=streaming_chunks_batch_size,json=streamingChunksBatchSize,proto3" json:"streaming_chunks_batch_size,omitempty"` } func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } @@ -117,6 +129,8 @@ type SeriesResponse struct { // *SeriesResponse_Warning // *SeriesResponse_Hints // *SeriesResponse_Stats + // *SeriesResponse_StreamingSeries + // *SeriesResponse_StreamingChunks Result isSeriesResponse_Result `protobuf_oneof:"result"` } @@ -171,11 +185,19 @@ type SeriesResponse_Hints struct { type SeriesResponse_Stats struct { Stats *Stats `protobuf:"bytes,4,opt,name=stats,proto3,oneof"` } +type SeriesResponse_StreamingSeries struct { + StreamingSeries *StreamingSeriesBatch `protobuf:"bytes,5,opt,name=streaming_series,json=streamingSeries,proto3,oneof"` +} +type SeriesResponse_StreamingChunks struct { + StreamingChunks *StreamingChunksBatch `protobuf:"bytes,6,opt,name=streaming_chunks,json=streamingChunks,proto3,oneof"` +} -func (*SeriesResponse_Series) isSeriesResponse_Result() {} -func (*SeriesResponse_Warning) isSeriesResponse_Result() {} -func (*SeriesResponse_Hints) isSeriesResponse_Result() {} -func (*SeriesResponse_Stats) isSeriesResponse_Result() {} +func (*SeriesResponse_Series) isSeriesResponse_Result() {} +func (*SeriesResponse_Warning) isSeriesResponse_Result() {} +func (*SeriesResponse_Hints) isSeriesResponse_Result() {} +func (*SeriesResponse_Stats) isSeriesResponse_Result() {} +func (*SeriesResponse_StreamingSeries) isSeriesResponse_Result() {} +func (*SeriesResponse_StreamingChunks) isSeriesResponse_Result() {} func (m *SeriesResponse) GetResult() isSeriesResponse_Result { if m != nil { @@ -212,6 +234,20 @@ func (m *SeriesResponse) GetStats() *Stats { return nil } +func (m *SeriesResponse) GetStreamingSeries() *StreamingSeriesBatch { + if x, ok := m.GetResult().(*SeriesResponse_StreamingSeries); ok { + return x.StreamingSeries + } + return nil +} + +func (m *SeriesResponse) GetStreamingChunks() *StreamingChunksBatch { + if x, ok := m.GetResult().(*SeriesResponse_StreamingChunks); ok { + return x.StreamingChunks + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*SeriesResponse) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -219,6 +255,8 @@ func (*SeriesResponse) XXX_OneofWrappers() []interface{} { (*SeriesResponse_Warning)(nil), (*SeriesResponse_Hints)(nil), (*SeriesResponse_Stats)(nil), + (*SeriesResponse_StreamingSeries)(nil), + (*SeriesResponse_StreamingChunks)(nil), } } @@ -402,51 +440,56 @@ func init() { func init() { proto.RegisterFile("rpc.proto", fileDescriptor_77a6da22d6a3feb1) } var fileDescriptor_77a6da22d6a3feb1 = []byte{ - // 692 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x93, 0x31, 0x6f, 0xd3, 0x40, - 0x14, 0xc7, 0x7d, 0xf1, 0xd9, 0xb9, 0x5c, 0x9a, 0xca, 0xbd, 0x96, 0xca, 0x35, 0xd2, 0x35, 0xb2, - 0x84, 0x14, 0x21, 0x70, 0x51, 0x91, 0x40, 0x8c, 0x4d, 0x25, 0x54, 0x2c, 0x60, 0x70, 0x11, 0x03, - 0x4b, 0xe4, 0xa4, 0xd7, 0xc4, 0x6a, 0x62, 0x07, 0x9f, 0x0d, 0xcd, 0xc6, 0x47, 0xe0, 0x63, 0x20, - 0x31, 0xf3, 0x05, 0x98, 0xba, 0xd1, 0xb1, 0x13, 0x22, 0xee, 0xc2, 0xd8, 0x99, 0x09, 0xf9, 0xce, - 0x69, 0x6a, 0x14, 0x54, 0x2a, 0xb1, 0xe5, 0xfd, 0xff, 0x2f, 0x77, 0xef, 0xff, 0x7b, 0x67, 0x5c, - 0x8b, 0xc7, 0x3d, 0x67, 0x1c, 0x47, 0x49, 0x44, 0xf4, 0x64, 0xe0, 0x87, 0x11, 0xb7, 0xea, 0xc9, - 0x64, 0xcc, 0xb8, 0x14, 0xad, 0xfb, 0xfd, 0x20, 0x19, 0xa4, 0x5d, 0xa7, 0x17, 0x8d, 0xb6, 0xfa, - 0x51, 0x3f, 0xda, 0x12, 0x72, 0x37, 0x3d, 0x14, 0x95, 0x28, 0xc4, 0xaf, 0xa2, 0x7d, 0xa3, 0x1f, - 0x45, 0xfd, 0x21, 0x9b, 0x77, 0xf9, 0xe1, 0x44, 0x5a, 0xf6, 0x2f, 0x80, 0x1b, 0xfb, 0x2c, 0x0e, - 0x18, 0xf7, 0xd8, 0xdb, 0x94, 0xf1, 0x84, 0x6c, 0x60, 0x34, 0x0a, 0xc2, 0x4e, 0x12, 0x8c, 0x98, - 0x09, 0x9a, 0xa0, 0xa5, 0x7a, 0xd5, 0x51, 0x10, 0xbe, 0x0a, 0x46, 0x4c, 0x58, 0xfe, 0xb1, 0xb4, - 0x2a, 0x85, 0xe5, 0x1f, 0x0b, 0xeb, 0x51, 0x6e, 0x25, 0xbd, 0x01, 0x8b, 0xb9, 0xa9, 0x36, 0xd5, - 0x56, 0x7d, 0x7b, 0xcd, 0x91, 0x93, 0x3b, 0xcf, 0xfd, 0x2e, 0x1b, 0xbe, 0x90, 0x66, 0x1b, 0x9e, - 0x7c, 0xdf, 0x54, 0xbc, 0xcb, 0x5e, 0xb2, 0x89, 0xeb, 0xfc, 0x28, 0x18, 0x77, 0x7a, 0x83, 0x34, - 0x3c, 0xe2, 0x26, 0x6a, 0x82, 0x16, 0xf2, 0x70, 0x2e, 0xed, 0x0a, 0x85, 0xdc, 0xc5, 0xda, 0x20, - 0x08, 0x13, 0x6e, 0xd6, 0x9a, 0x40, 0x9c, 0x2a, 0xb3, 0x38, 0xb3, 0x2c, 0xce, 0x4e, 0x38, 0xf1, - 0x64, 0x8b, 0x0b, 0x11, 0x34, 0x34, 0x17, 0x22, 0xcd, 0xd0, 0x5d, 0x88, 0x74, 0xa3, 0xea, 0x42, - 0x54, 0x35, 0x90, 0x0b, 0x11, 0x36, 0xea, 0x2e, 0x44, 0x75, 0x63, 0xc9, 0x85, 0x68, 0xc9, 0x68, - 0xb8, 0x10, 0x35, 0x8c, 0x65, 0xfb, 0x31, 0xd6, 0xf6, 0x13, 0x3f, 0xe1, 0xc4, 0xc1, 0xab, 0x87, - 0x2c, 0x9f, 0xe8, 0xa0, 0x13, 0x84, 0x07, 0xec, 0xb8, 0xd3, 0x9d, 0x24, 0x8c, 0x8b, 0xf8, 0xd0, - 0x5b, 0x29, 0xac, 0x67, 0xb9, 0xd3, 0xce, 0x0d, 0xfb, 0x0b, 0xc0, 0xcb, 0x33, 0x6a, 0x7c, 0x1c, - 0x85, 0x9c, 0x91, 0x16, 0xd6, 0xb9, 0x50, 0xc4, 0xbf, 0xea, 0xdb, 0xcb, 0xb3, 0xf8, 0xb2, 0x6f, - 0x4f, 0xf1, 0x0a, 0x9f, 0x58, 0xb8, 0xfa, 0xde, 0x8f, 0xc3, 0x20, 0xec, 0x0b, 0x88, 0xb5, 0x3d, - 0xc5, 0x9b, 0x09, 0xe4, 0xde, 0x2c, 0xad, 0xfa, 0xf7, 0xb4, 0x7b, 0x4a, 0x91, 0x97, 0xdc, 0xc1, - 0x1a, 0xcf, 0xe7, 0x37, 0xa1, 0xe8, 0x6e, 0x5c, 0x5e, 0x99, 0x8b, 0x79, 0x9b, 0x70, 0xdb, 0x08, - 0xeb, 0x31, 0xe3, 0xe9, 0x30, 0xb1, 0x3f, 0x03, 0xbc, 0x22, 0xd6, 0xf1, 0xd2, 0x1f, 0xcd, 0x37, - 0xbe, 0x26, 0x8e, 0x89, 0x13, 0x71, 0xa9, 0xea, 0xc9, 0x82, 0x18, 0x58, 0x65, 0xe1, 0x81, 0x38, - 0x5a, 0xf5, 0xf2, 0x9f, 0xf3, 0x55, 0x68, 0xd7, 0xae, 0xa2, 0xf4, 0x1e, 0xf4, 0x7f, 0x7f, 0x0f, - 0x2e, 0x44, 0xc0, 0xa8, 0xb8, 0x10, 0x55, 0x0c, 0xd5, 0x8e, 0x31, 0xb9, 0x3a, 0x6c, 0x01, 0x7a, - 0x0d, 0x6b, 0x61, 0x2e, 0x98, 0xa0, 0xa9, 0xb6, 0x6a, 0x9e, 0x2c, 0x88, 0x85, 0x51, 0xc1, 0x90, - 0x9b, 0x15, 0x61, 0x5c, 0xd6, 0xf3, 0xb9, 0xd5, 0x6b, 0xe7, 0xb6, 0xbf, 0x82, 0xe2, 0xd2, 0xd7, - 0xfe, 0x30, 0x2d, 0x21, 0x1a, 0xe6, 0xaa, 0x58, 0x6e, 0xcd, 0x93, 0xc5, 0x1c, 0x1c, 0x5c, 0x00, - 0x4e, 0x5b, 0x00, 0x4e, 0xbf, 0x19, 0xb8, 0xea, 0x8d, 0xc0, 0x55, 0x0c, 0xd5, 0x85, 0x48, 0x35, - 0xa0, 0x9d, 0xe2, 0xd5, 0x52, 0x86, 0x82, 0xdc, 0x3a, 0xd6, 0xdf, 0x09, 0xa5, 0x40, 0x57, 0x54, - 0xff, 0x8b, 0xdd, 0xf6, 0x37, 0x90, 0x7f, 0x4f, 0x51, 0xcc, 0xc8, 0x13, 0xac, 0xcb, 0x67, 0x4f, - 0x6e, 0x95, 0x3f, 0x83, 0x82, 0xa7, 0xb5, 0xfe, 0xa7, 0x2c, 0x47, 0x7c, 0x00, 0xc8, 0x2e, 0xc6, - 0xf3, 0xa5, 0x93, 0x8d, 0x52, 0xf6, 0xab, 0xaf, 0xd6, 0xb2, 0x16, 0x59, 0x45, 0xd2, 0xa7, 0xb8, - 0x7e, 0x05, 0x00, 0x29, 0xb7, 0x96, 0x36, 0x6b, 0xdd, 0x5e, 0xe8, 0xc9, 0x73, 0xda, 0x3b, 0x27, - 0x53, 0xaa, 0x9c, 0x4e, 0xa9, 0x72, 0x36, 0xa5, 0xca, 0xc5, 0x94, 0x82, 0x0f, 0x19, 0x05, 0x9f, - 0x32, 0x0a, 0x4e, 0x32, 0x0a, 0x4e, 0x33, 0x0a, 0x7e, 0x64, 0x14, 0xfc, 0xcc, 0xa8, 0x72, 0x91, - 0x51, 0xf0, 0xf1, 0x9c, 0x2a, 0xa7, 0xe7, 0x54, 0x39, 0x3b, 0xa7, 0xca, 0x9b, 0x2a, 0xcf, 0x41, - 0x8c, 0xbb, 0x5d, 0x5d, 0x90, 0x7a, 0xf8, 0x3b, 0x00, 0x00, 0xff, 0xff, 0xe8, 0x55, 0x20, 0x90, - 0xd3, 0x05, 0x00, 0x00, + // 772 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x93, 0x41, 0x6f, 0xd3, 0x48, + 0x14, 0xc7, 0x3d, 0xf1, 0xd8, 0x99, 0x4c, 0x9a, 0xae, 0x3b, 0xed, 0x56, 0xae, 0xbb, 0x72, 0xa3, + 0x48, 0x2b, 0x45, 0xab, 0xdd, 0x74, 0x55, 0x24, 0x10, 0x07, 0x0e, 0x4d, 0x25, 0x94, 0x5a, 0xc0, + 0xc1, 0x45, 0x1c, 0xb8, 0x44, 0x4e, 0x32, 0x4d, 0xac, 0x26, 0x76, 0xf0, 0x38, 0xd0, 0xf4, 0xc4, + 0x47, 0xe0, 0x63, 0x20, 0xf1, 0x09, 0xb8, 0x72, 0xaa, 0xc4, 0x81, 0x1e, 0x7b, 0x42, 0x24, 0xbd, + 0x70, 0xec, 0x47, 0x40, 0x9e, 0x99, 0xc4, 0x31, 0x4a, 0x55, 0x2a, 0x71, 0xf3, 0xfb, 0xff, 0xdf, + 0xcc, 0xbc, 0xf7, 0x7b, 0xcf, 0xb8, 0x10, 0x0d, 0xdb, 0xb5, 0x61, 0x14, 0xc6, 0x21, 0xd1, 0xe3, + 0x9e, 0x17, 0x84, 0xcc, 0x2a, 0xc6, 0xe3, 0x21, 0x65, 0x42, 0xb4, 0xfe, 0xeb, 0xfa, 0x71, 0x6f, + 0xd4, 0xaa, 0xb5, 0xc3, 0xc1, 0x6e, 0x37, 0xec, 0x86, 0xbb, 0x5c, 0x6e, 0x8d, 0x8e, 0x79, 0xc4, + 0x03, 0xfe, 0x25, 0xd3, 0xb7, 0xba, 0x61, 0xd8, 0xed, 0xd3, 0x34, 0xcb, 0x0b, 0xc6, 0xc2, 0xaa, + 0x7c, 0xcc, 0xe1, 0xd2, 0x11, 0x8d, 0x7c, 0xca, 0x5c, 0xfa, 0x6a, 0x44, 0x59, 0x4c, 0xb6, 0x30, + 0x1a, 0xf8, 0x41, 0x33, 0xf6, 0x07, 0xd4, 0x04, 0x65, 0x50, 0x55, 0xdd, 0xfc, 0xc0, 0x0f, 0x9e, + 0xfb, 0x03, 0xca, 0x2d, 0xef, 0x54, 0x58, 0x39, 0x69, 0x79, 0xa7, 0xdc, 0xba, 0x9f, 0x58, 0x71, + 0xbb, 0x47, 0x23, 0x66, 0xaa, 0x65, 0xb5, 0x5a, 0xdc, 0xdb, 0xa8, 0x89, 0xca, 0x6b, 0x4f, 0xbc, + 0x16, 0xed, 0x3f, 0x15, 0x66, 0x1d, 0x9e, 0x7f, 0xdd, 0x51, 0xdc, 0x79, 0x2e, 0xd9, 0xc1, 0x45, + 0x76, 0xe2, 0x0f, 0x9b, 0xed, 0xde, 0x28, 0x38, 0x61, 0x26, 0x2a, 0x83, 0x2a, 0x72, 0x71, 0x22, + 0x1d, 0x70, 0x85, 0xfc, 0x83, 0xb5, 0x9e, 0x1f, 0xc4, 0xcc, 0x2c, 0x94, 0x01, 0xbf, 0x55, 0xf4, + 0x52, 0x9b, 0xf5, 0x52, 0xdb, 0x0f, 0xc6, 0xae, 0x48, 0x21, 0x8f, 0xf0, 0x36, 0x8b, 0x23, 0xea, + 0x0d, 0xfc, 0xa0, 0x2b, 0x6f, 0x6c, 0xb6, 0x92, 0x97, 0x9a, 0xcc, 0x3f, 0xa3, 0x66, 0xa7, 0x0c, + 0xaa, 0xd0, 0x35, 0xe7, 0x29, 0xe2, 0x85, 0x7a, 0x92, 0x70, 0xe4, 0x9f, 0x51, 0x07, 0x22, 0x68, + 0x68, 0x0e, 0x44, 0x9a, 0xa1, 0x3b, 0x10, 0xe9, 0x46, 0xde, 0x81, 0x28, 0x6f, 0x20, 0x07, 0x22, + 0x6c, 0x14, 0x1d, 0x88, 0x8a, 0xc6, 0x8a, 0x03, 0xd1, 0x8a, 0x51, 0x72, 0x20, 0x2a, 0x19, 0xab, + 0x95, 0x07, 0x58, 0x3b, 0x8a, 0xbd, 0x98, 0x91, 0x1a, 0x5e, 0x3f, 0xa6, 0x49, 0x43, 0x9d, 0xa6, + 0x1f, 0x74, 0xe8, 0x69, 0xb3, 0x35, 0x8e, 0x29, 0xe3, 0xf4, 0xa0, 0xbb, 0x26, 0xad, 0xc3, 0xc4, + 0xa9, 0x27, 0x46, 0xe5, 0x73, 0x0e, 0xaf, 0xce, 0xa0, 0xb3, 0x61, 0x18, 0x30, 0x4a, 0xaa, 0x58, + 0x67, 0x5c, 0xe1, 0xa7, 0x8a, 0x7b, 0xab, 0x33, 0x7a, 0x22, 0xaf, 0xa1, 0xb8, 0xd2, 0x27, 0x16, + 0xce, 0xbf, 0xf1, 0xa2, 0xc0, 0x0f, 0xba, 0x7c, 0x06, 0x85, 0x86, 0xe2, 0xce, 0x04, 0xf2, 0xef, + 0x0c, 0x96, 0x7a, 0x33, 0xac, 0x86, 0x32, 0xc3, 0xf5, 0x37, 0xd6, 0x58, 0x52, 0xbf, 0x09, 0x79, + 0x76, 0x69, 0xfe, 0x64, 0x22, 0x26, 0x69, 0xdc, 0x25, 0x87, 0xd8, 0x48, 0xa9, 0xca, 0x22, 0x35, + 0x7e, 0xe2, 0xaf, 0xf4, 0x84, 0xf4, 0x45, 0xb5, 0x1c, 0x69, 0x43, 0x71, 0xff, 0x60, 0x59, 0x3d, + 0x7b, 0x95, 0x1c, 0xb9, 0x7e, 0xc3, 0x55, 0x0b, 0xd3, 0xc9, 0x5c, 0x25, 0x75, 0x84, 0xf5, 0x88, + 0xb2, 0x51, 0x3f, 0xae, 0x7c, 0x00, 0x78, 0x8d, 0xef, 0xd8, 0x33, 0x6f, 0x90, 0xae, 0xf1, 0x06, + 0x6f, 0x2e, 0x8a, 0x39, 0x0a, 0xd5, 0x15, 0x01, 0x31, 0xb0, 0x4a, 0x83, 0x0e, 0x6f, 0x58, 0x75, + 0x93, 0xcf, 0x74, 0xbf, 0xb4, 0xdb, 0xf7, 0x6b, 0x71, 0xc9, 0xf5, 0x5f, 0x5f, 0x72, 0x07, 0x22, + 0x60, 0xe4, 0x1c, 0x88, 0x72, 0x86, 0x5a, 0x89, 0x30, 0x59, 0x2c, 0x56, 0x8e, 0x7f, 0x03, 0x6b, + 0x41, 0x22, 0x98, 0xa0, 0xac, 0x56, 0x0b, 0xae, 0x08, 0x88, 0x85, 0x91, 0x9c, 0x2c, 0x33, 0x73, + 0xdc, 0x98, 0xc7, 0x69, 0xdd, 0xea, 0xad, 0x75, 0x57, 0x3e, 0x01, 0xf9, 0xe8, 0x0b, 0xaf, 0x3f, + 0xca, 0x20, 0xea, 0x27, 0x2a, 0x5f, 0xb9, 0x82, 0x2b, 0x82, 0x14, 0x1c, 0x5c, 0x02, 0x4e, 0x5b, + 0x02, 0x4e, 0xbf, 0x1b, 0xb8, 0xfc, 0x9d, 0xc0, 0xe5, 0x0c, 0xd5, 0x81, 0x48, 0x35, 0x60, 0x65, + 0x84, 0xd7, 0x33, 0x3d, 0x48, 0x72, 0x9b, 0x58, 0x7f, 0xcd, 0x15, 0x89, 0x4e, 0x46, 0xbf, 0x8b, + 0xdd, 0xde, 0x17, 0x90, 0xfc, 0xe5, 0x61, 0x44, 0xc9, 0x43, 0xac, 0xcb, 0x35, 0xfe, 0x33, 0xfb, + 0x73, 0x4a, 0x9e, 0xd6, 0xe6, 0xcf, 0xb2, 0x28, 0xf1, 0x7f, 0x40, 0x0e, 0x30, 0x4e, 0x87, 0x4e, + 0xb6, 0x32, 0xbd, 0x2f, 0x6e, 0xad, 0x65, 0x2d, 0xb3, 0x64, 0xa7, 0x8f, 0x71, 0x71, 0x01, 0x00, + 0xc9, 0xa6, 0x66, 0x26, 0x6b, 0x6d, 0x2f, 0xf5, 0xc4, 0x3d, 0xf5, 0xfd, 0xf3, 0x89, 0xad, 0x5c, + 0x4c, 0x6c, 0xe5, 0x72, 0x62, 0x2b, 0xd7, 0x13, 0x1b, 0xbc, 0x9d, 0xda, 0xe0, 0xfd, 0xd4, 0x06, + 0xe7, 0x53, 0x1b, 0x5c, 0x4c, 0x6d, 0xf0, 0x6d, 0x6a, 0x83, 0xef, 0x53, 0x5b, 0xb9, 0x9e, 0xda, + 0xe0, 0xdd, 0x95, 0xad, 0x5c, 0x5c, 0xd9, 0xca, 0xe5, 0x95, 0xad, 0xbc, 0xcc, 0xb3, 0x04, 0xc4, + 0xb0, 0xd5, 0xd2, 0x39, 0xa9, 0x7b, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x9b, 0x71, 0x84, 0x02, + 0xa8, 0x06, 0x00, 0x00, } func (this *SeriesRequest) Equal(that interface{}) bool { @@ -488,6 +531,9 @@ func (this *SeriesRequest) Equal(that interface{}) bool { if !this.Hints.Equal(that1.Hints) { return false } + if this.StreamingChunksBatchSize != that1.StreamingChunksBatchSize { + return false + } return true } func (this *Stats) Equal(that interface{}) bool { @@ -640,6 +686,54 @@ func (this *SeriesResponse_Stats) Equal(that interface{}) bool { } return true } +func (this *SeriesResponse_StreamingSeries) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse_StreamingSeries) + if !ok { + that2, ok := that.(SeriesResponse_StreamingSeries) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.StreamingSeries.Equal(that1.StreamingSeries) { + return false + } + return true +} +func (this *SeriesResponse_StreamingChunks) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SeriesResponse_StreamingChunks) + if !ok { + that2, ok := that.(SeriesResponse_StreamingChunks) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.StreamingChunks.Equal(that1.StreamingChunks) { + return false + } + return true +} func (this *LabelNamesRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -803,7 +897,7 @@ func (this *SeriesRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 10) s = append(s, "&storepb.SeriesRequest{") s = append(s, "MinTime: "+fmt.Sprintf("%#v", this.MinTime)+",\n") s = append(s, "MaxTime: "+fmt.Sprintf("%#v", this.MaxTime)+",\n") @@ -818,6 +912,7 @@ func (this *SeriesRequest) GoString() string { if this.Hints != nil { s = append(s, "Hints: "+fmt.Sprintf("%#v", this.Hints)+",\n") } + s = append(s, "StreamingChunksBatchSize: "+fmt.Sprintf("%#v", this.StreamingChunksBatchSize)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -835,7 +930,7 @@ func (this *SeriesResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 8) + s := make([]string, 0, 10) s = append(s, "&storepb.SeriesResponse{") if this.Result != nil { s = append(s, "Result: "+fmt.Sprintf("%#v", this.Result)+",\n") @@ -875,6 +970,22 @@ func (this *SeriesResponse_Stats) GoString() string { `Stats:` + fmt.Sprintf("%#v", this.Stats) + `}`}, ", ") return s } +func (this *SeriesResponse_StreamingSeries) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SeriesResponse_StreamingSeries{` + + `StreamingSeries:` + fmt.Sprintf("%#v", this.StreamingSeries) + `}`}, ", ") + return s +} +func (this *SeriesResponse_StreamingChunks) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&storepb.SeriesResponse_StreamingChunks{` + + `StreamingChunks:` + fmt.Sprintf("%#v", this.StreamingChunks) + `}`}, ", ") + return s +} func (this *LabelNamesRequest) GoString() string { if this == nil { return "nil" @@ -1177,6 +1288,13 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.StreamingChunksBatchSize != 0 { + i = encodeVarintRpc(dAtA, i, uint64(m.StreamingChunksBatchSize)) + i-- + dAtA[i] = 0x6 + i-- + dAtA[i] = 0xa0 + } if m.Hints != nil { { size, err := m.Hints.MarshalToSizedBuffer(dAtA[:i]) @@ -1359,6 +1477,46 @@ func (m *SeriesResponse_Stats) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *SeriesResponse_StreamingSeries) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SeriesResponse_StreamingSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.StreamingSeries != nil { + { + size, err := m.StreamingSeries.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + return len(dAtA) - i, nil +} +func (m *SeriesResponse_StreamingChunks) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *SeriesResponse_StreamingChunks) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.StreamingChunks != nil { + { + size, err := m.StreamingChunks.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRpc(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} func (m *LabelNamesRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1626,6 +1784,9 @@ func (m *SeriesRequest) Size() (n int) { l = m.Hints.Size() n += 1 + l + sovRpc(uint64(l)) } + if m.StreamingChunksBatchSize != 0 { + n += 2 + sovRpc(uint64(m.StreamingChunksBatchSize)) + } return n } @@ -1699,6 +1860,30 @@ func (m *SeriesResponse_Stats) Size() (n int) { } return n } +func (m *SeriesResponse_StreamingSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.StreamingSeries != nil { + l = m.StreamingSeries.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} +func (m *SeriesResponse_StreamingChunks) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.StreamingChunks != nil { + l = m.StreamingChunks.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} func (m *LabelNamesRequest) Size() (n int) { if m == nil { return 0 @@ -1824,6 +2009,7 @@ func (this *SeriesRequest) String() string { `Matchers:` + repeatedStringForMatchers + `,`, `SkipChunks:` + fmt.Sprintf("%v", this.SkipChunks) + `,`, `Hints:` + strings.Replace(fmt.Sprintf("%v", this.Hints), "Any", "types.Any", 1) + `,`, + `StreamingChunksBatchSize:` + fmt.Sprintf("%v", this.StreamingChunksBatchSize) + `,`, `}`, }, "") return s @@ -1888,6 +2074,26 @@ func (this *SeriesResponse_Stats) String() string { }, "") return s } +func (this *SeriesResponse_StreamingSeries) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse_StreamingSeries{`, + `StreamingSeries:` + strings.Replace(fmt.Sprintf("%v", this.StreamingSeries), "StreamingSeriesBatch", "StreamingSeriesBatch", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SeriesResponse_StreamingChunks) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SeriesResponse_StreamingChunks{`, + `StreamingChunks:` + strings.Replace(fmt.Sprintf("%v", this.StreamingChunks), "StreamingChunksBatch", "StreamingChunksBatch", 1) + `,`, + `}`, + }, "") + return s +} func (this *LabelNamesRequest) String() string { if this == nil { return "nil" @@ -2114,6 +2320,25 @@ func (m *SeriesRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 100: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamingChunksBatchSize", wireType) + } + m.StreamingChunksBatchSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StreamingChunksBatchSize |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) @@ -2376,6 +2601,76 @@ func (m *SeriesResponse) Unmarshal(dAtA []byte) error { } m.Result = &SeriesResponse_Stats{v} iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamingSeries", 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 := &StreamingSeriesBatch{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SeriesResponse_StreamingSeries{v} + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamingChunks", 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 := &StreamingChunksBatch{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &SeriesResponse_StreamingChunks{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/storegateway/storepb/rpc.proto b/pkg/storegateway/storepb/rpc.proto index 915753cacde..dc6c9cd09b1 100644 --- a/pkg/storegateway/storepb/rpc.proto +++ b/pkg/storegateway/storepb/rpc.proto @@ -79,6 +79,19 @@ message SeriesRequest { // Thanos shard_info. reserved 13; + + // If streaming_chunks_batch_size=0, the response must only contain one 'series' at a time + // with the series labels and chunks data sent together. + // If streaming_chunks_batch_size > 0 + // - The store may choose to send the streaming_series/streaming_chunks OR behave as + // if streaming_chunks_batch_size=0 if it does not support streaming series. + // - The store must not send a mix of 'series' and streaming_series/streaming_chunks for a single request. + // - If the store chooses to send streaming series, all the streaming_series must be sent before + // sending any streaming_chunks, with the last streaming_series response containing is_end_of_series_stream=true. + // The order of series in both streaming_series/streaming_chunks must match and the size of the batch must not + // cross streaming_chunks_batch_size, although it can be lower than that. + // The proto field ID is 100 so that we have an option to bring back compatibility with Thanos' storage API. + uint64 streaming_chunks_batch_size = 100; } message Stats { @@ -105,6 +118,18 @@ message SeriesResponse { /// stats is a object containing stats for a series response from the store-gateways so that we can collect stats /// related to the processing the series response on store-gateways did available to the querier and query-frontends. Stats stats = 4; + + /// streaming_series is a list of series labels sent as part of a streaming Series call. + /// These are populated only when streaming_chunks_batch_size > 0 in the series request. + /// Series are sent in batches because sending one at a time has additional CPU overhead for not much memory gains. + StreamingSeriesBatch streaming_series = 5; + + /// streaming_chunks is a list of chunks sent as part of a streaming Series request. + /// They are associated with series labels sent as streaming_series earlier in the same Series request. + /// These are populated only when streaming_chunks_batch_size > 0 in the series request. + /// Chunks are sent in batches because sending one series' chunks at a time has additional + // CPU overhead for not much memory gains. + StreamingChunksBatch streaming_chunks = 6; } } diff --git a/pkg/storegateway/storepb/types.pb.go b/pkg/storegateway/storepb/types.pb.go index 4a53330df89..542c4cb611d 100644 --- a/pkg/storegateway/storepb/types.pb.go +++ b/pkg/storegateway/storepb/types.pb.go @@ -76,7 +76,7 @@ var LabelMatcher_Type_value = map[string]int32{ } func (LabelMatcher_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_d938547f84707355, []int{3, 0} + return fileDescriptor_d938547f84707355, []int{7, 0} } type Chunk struct { @@ -153,6 +153,152 @@ func (m *Series) XXX_DiscardUnknown() { var xxx_messageInfo_Series proto.InternalMessageInfo +type StreamingSeries struct { + Labels []github_com_grafana_mimir_pkg_mimirpb.LabelAdapter `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/grafana/mimir/pkg/mimirpb.LabelAdapter" json:"labels"` +} + +func (m *StreamingSeries) Reset() { *m = StreamingSeries{} } +func (*StreamingSeries) ProtoMessage() {} +func (*StreamingSeries) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{2} +} +func (m *StreamingSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamingSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StreamingSeries.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 *StreamingSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamingSeries.Merge(m, src) +} +func (m *StreamingSeries) XXX_Size() int { + return m.Size() +} +func (m *StreamingSeries) XXX_DiscardUnknown() { + xxx_messageInfo_StreamingSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamingSeries proto.InternalMessageInfo + +type StreamingSeriesBatch struct { + Series []*StreamingSeries `protobuf:"bytes,1,rep,name=series,proto3" json:"series,omitempty"` + IsEndOfSeriesStream bool `protobuf:"varint,2,opt,name=is_end_of_series_stream,json=isEndOfSeriesStream,proto3" json:"is_end_of_series_stream,omitempty"` +} + +func (m *StreamingSeriesBatch) Reset() { *m = StreamingSeriesBatch{} } +func (*StreamingSeriesBatch) ProtoMessage() {} +func (*StreamingSeriesBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{3} +} +func (m *StreamingSeriesBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamingSeriesBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StreamingSeriesBatch.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 *StreamingSeriesBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamingSeriesBatch.Merge(m, src) +} +func (m *StreamingSeriesBatch) XXX_Size() int { + return m.Size() +} +func (m *StreamingSeriesBatch) XXX_DiscardUnknown() { + xxx_messageInfo_StreamingSeriesBatch.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamingSeriesBatch proto.InternalMessageInfo + +type StreamingChunks struct { + SeriesIndex uint64 `protobuf:"varint,1,opt,name=series_index,json=seriesIndex,proto3" json:"series_index,omitempty"` + Chunks []AggrChunk `protobuf:"bytes,2,rep,name=chunks,proto3" json:"chunks"` +} + +func (m *StreamingChunks) Reset() { *m = StreamingChunks{} } +func (*StreamingChunks) ProtoMessage() {} +func (*StreamingChunks) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{4} +} +func (m *StreamingChunks) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamingChunks) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StreamingChunks.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 *StreamingChunks) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamingChunks.Merge(m, src) +} +func (m *StreamingChunks) XXX_Size() int { + return m.Size() +} +func (m *StreamingChunks) XXX_DiscardUnknown() { + xxx_messageInfo_StreamingChunks.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamingChunks proto.InternalMessageInfo + +type StreamingChunksBatch struct { + Series []*StreamingChunks `protobuf:"bytes,1,rep,name=series,proto3" json:"series,omitempty"` +} + +func (m *StreamingChunksBatch) Reset() { *m = StreamingChunksBatch{} } +func (*StreamingChunksBatch) ProtoMessage() {} +func (*StreamingChunksBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_d938547f84707355, []int{5} +} +func (m *StreamingChunksBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamingChunksBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StreamingChunksBatch.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 *StreamingChunksBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamingChunksBatch.Merge(m, src) +} +func (m *StreamingChunksBatch) XXX_Size() int { + return m.Size() +} +func (m *StreamingChunksBatch) XXX_DiscardUnknown() { + xxx_messageInfo_StreamingChunksBatch.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamingChunksBatch proto.InternalMessageInfo + type AggrChunk struct { MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` @@ -162,7 +308,7 @@ type AggrChunk struct { func (m *AggrChunk) Reset() { *m = AggrChunk{} } func (*AggrChunk) ProtoMessage() {} func (*AggrChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_d938547f84707355, []int{2} + return fileDescriptor_d938547f84707355, []int{6} } func (m *AggrChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -201,7 +347,7 @@ type LabelMatcher struct { func (m *LabelMatcher) Reset() { *m = LabelMatcher{} } func (*LabelMatcher) ProtoMessage() {} func (*LabelMatcher) Descriptor() ([]byte, []int) { - return fileDescriptor_d938547f84707355, []int{3} + return fileDescriptor_d938547f84707355, []int{7} } func (m *LabelMatcher) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -235,6 +381,10 @@ func init() { proto.RegisterEnum("thanos.LabelMatcher_Type", LabelMatcher_Type_name, LabelMatcher_Type_value) proto.RegisterType((*Chunk)(nil), "thanos.Chunk") proto.RegisterType((*Series)(nil), "thanos.Series") + proto.RegisterType((*StreamingSeries)(nil), "thanos.StreamingSeries") + proto.RegisterType((*StreamingSeriesBatch)(nil), "thanos.StreamingSeriesBatch") + proto.RegisterType((*StreamingChunks)(nil), "thanos.StreamingChunks") + proto.RegisterType((*StreamingChunksBatch)(nil), "thanos.StreamingChunksBatch") proto.RegisterType((*AggrChunk)(nil), "thanos.AggrChunk") proto.RegisterType((*LabelMatcher)(nil), "thanos.LabelMatcher") } @@ -242,43 +392,50 @@ func init() { func init() { proto.RegisterFile("types.proto", fileDescriptor_d938547f84707355) } var fileDescriptor_d938547f84707355 = []byte{ - // 567 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xcd, 0x6e, 0xd3, 0x40, - 0x10, 0xf6, 0x26, 0x8e, 0xe3, 0x6c, 0x5b, 0x58, 0xb6, 0x15, 0x72, 0x7b, 0xd8, 0x46, 0x3e, 0x45, - 0x48, 0x75, 0xa0, 0x70, 0x41, 0xe2, 0xd2, 0xa0, 0xa0, 0x2a, 0xe2, 0xa7, 0x75, 0x8b, 0x84, 0x10, - 0x52, 0xb5, 0x76, 0x37, 0xce, 0xaa, 0xf1, 0x8f, 0xd6, 0x1b, 0x68, 0x6f, 0x7d, 0x04, 0x5e, 0x81, - 0x1b, 0x2f, 0x82, 0x94, 0x1b, 0x39, 0x56, 0x1c, 0x2a, 0xe2, 0x5c, 0x38, 0xf6, 0x11, 0x90, 0xd7, - 0x0e, 0x24, 0xea, 0xa5, 0x27, 0xcf, 0xcc, 0xf7, 0xcd, 0x7c, 0x9f, 0x67, 0x6c, 0xb8, 0x22, 0x2f, - 0x12, 0x96, 0x3a, 0x89, 0x88, 0x65, 0x8c, 0x0d, 0x39, 0xa0, 0x51, 0x9c, 0x6e, 0xed, 0x04, 0x5c, - 0x0e, 0x46, 0x9e, 0xe3, 0xc7, 0x61, 0x3b, 0x88, 0x83, 0xb8, 0xad, 0x60, 0x6f, 0xd4, 0x57, 0x99, - 0x4a, 0x54, 0x54, 0xb4, 0x6d, 0x3d, 0x5e, 0xa4, 0x0b, 0xda, 0xa7, 0x11, 0x6d, 0x87, 0x3c, 0xe4, - 0xa2, 0x9d, 0x9c, 0x05, 0x45, 0x94, 0x78, 0xc5, 0xb3, 0xe8, 0xb0, 0x7f, 0x02, 0x58, 0x7b, 0x39, - 0x18, 0x45, 0x67, 0xf8, 0x11, 0xd4, 0x73, 0x07, 0x16, 0x68, 0x82, 0xd6, 0xbd, 0xdd, 0x87, 0x4e, - 0xe1, 0xc0, 0x51, 0xa0, 0xd3, 0x8d, 0xfc, 0xf8, 0x94, 0x47, 0x81, 0xab, 0x38, 0xf8, 0x00, 0xea, - 0xa7, 0x54, 0x52, 0xab, 0xd2, 0x04, 0xad, 0xd5, 0xce, 0x8b, 0xf1, 0xf5, 0xb6, 0xf6, 0xeb, 0x7a, - 0xfb, 0xd9, 0x5d, 0xd4, 0x9d, 0xf7, 0x51, 0x4a, 0xfb, 0xac, 0x73, 0x21, 0xd9, 0xd1, 0x90, 0xfb, - 0xcc, 0x55, 0x93, 0xec, 0x7d, 0x68, 0xce, 0x35, 0xf0, 0x1a, 0x6c, 0x28, 0xd5, 0x93, 0x0f, 0xef, - 0x5c, 0xa4, 0xe1, 0x75, 0x78, 0xbf, 0x48, 0xf7, 0x79, 0x2a, 0xe3, 0x40, 0xd0, 0x10, 0x01, 0x6c, - 0xc1, 0x8d, 0xa2, 0xf8, 0x6a, 0x18, 0x53, 0xf9, 0x1f, 0xa9, 0xd8, 0xdf, 0x00, 0x34, 0x8e, 0x98, - 0xe0, 0x2c, 0xc5, 0x7d, 0x68, 0x0c, 0xa9, 0xc7, 0x86, 0xa9, 0x05, 0x9a, 0xd5, 0xd6, 0xca, 0xee, - 0xba, 0xe3, 0xc7, 0x42, 0xb2, 0xf3, 0xc4, 0x73, 0x5e, 0xe7, 0xf5, 0x03, 0xca, 0x45, 0xe7, 0x79, - 0xe9, 0xfe, 0xc9, 0x9d, 0xdc, 0xab, 0xbe, 0xbd, 0x53, 0x9a, 0x48, 0x26, 0xdc, 0x72, 0x3a, 0x6e, - 0x43, 0xc3, 0xcf, 0xcd, 0xa4, 0x56, 0x45, 0xe9, 0x3c, 0x98, 0x2f, 0x6f, 0x2f, 0x08, 0x84, 0xb2, - 0xd9, 0xd1, 0x73, 0x15, 0xb7, 0xa4, 0xd9, 0x97, 0x00, 0x36, 0xfe, 0x61, 0x78, 0x13, 0x9a, 0x21, - 0x8f, 0x4e, 0x24, 0x0f, 0x8b, 0xed, 0x57, 0xdd, 0x7a, 0xc8, 0xa3, 0x63, 0x1e, 0x32, 0x05, 0xd1, - 0xf3, 0x02, 0xaa, 0x94, 0x10, 0x3d, 0x57, 0xd0, 0x36, 0xac, 0x0a, 0xfa, 0xc5, 0xaa, 0x36, 0x41, - 0x6b, 0x65, 0x77, 0x6d, 0xe9, 0x5c, 0x6e, 0x8e, 0xf4, 0x74, 0x53, 0x47, 0xb5, 0x9e, 0x6e, 0xd6, - 0x90, 0xd1, 0xd3, 0x4d, 0x03, 0xd5, 0x7b, 0xba, 0x59, 0x47, 0x66, 0x4f, 0x37, 0x4d, 0xd4, 0xb0, - 0x7f, 0x00, 0xb8, 0xaa, 0x5e, 0xe6, 0x0d, 0x95, 0xfe, 0x80, 0x09, 0xbc, 0xb3, 0x74, 0xff, 0xcd, - 0xf9, 0xc0, 0x45, 0x8e, 0x73, 0x7c, 0x91, 0xb0, 0xf2, 0x13, 0xc0, 0x50, 0x8f, 0x68, 0xe9, 0xaa, - 0xe1, 0xaa, 0x18, 0x6f, 0xc0, 0xda, 0x67, 0x3a, 0x1c, 0x31, 0x65, 0xaa, 0xe1, 0x16, 0x89, 0xfd, - 0x09, 0xea, 0x79, 0x5f, 0x7e, 0xc7, 0xc5, 0x61, 0x27, 0xdd, 0x43, 0xa4, 0xe1, 0x0d, 0x88, 0x96, - 0x8a, 0x6f, 0xbb, 0x87, 0x08, 0xdc, 0xa2, 0xba, 0x5d, 0x54, 0xb9, 0x4d, 0x75, 0xbb, 0xa8, 0xda, - 0xd9, 0x1b, 0x4f, 0x89, 0x36, 0x99, 0x12, 0xed, 0x6a, 0x4a, 0xb4, 0x9b, 0x29, 0x01, 0x97, 0x19, - 0x01, 0xdf, 0x33, 0x02, 0xc6, 0x19, 0x01, 0x93, 0x8c, 0x80, 0xdf, 0x19, 0x01, 0x7f, 0x32, 0xa2, - 0xdd, 0x64, 0x04, 0x7c, 0x9d, 0x11, 0x6d, 0x32, 0x23, 0xda, 0xd5, 0x8c, 0x68, 0x1f, 0xeb, 0xa9, - 0x8c, 0x05, 0x4b, 0x3c, 0xcf, 0x50, 0xbf, 0xc2, 0xd3, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x08, - 0x77, 0xfb, 0x5f, 0x82, 0x03, 0x00, 0x00, + // 677 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x54, 0xbd, 0x6e, 0x13, 0x4d, + 0x14, 0xdd, 0xb1, 0xd7, 0xf6, 0x7a, 0x9c, 0x7c, 0xd9, 0x6f, 0x62, 0x11, 0x27, 0xc5, 0xc4, 0x6c, + 0x65, 0x21, 0x65, 0x0d, 0x21, 0x0d, 0x12, 0x4d, 0x8c, 0x0c, 0xc1, 0x02, 0x92, 0x4c, 0x82, 0x84, + 0x10, 0x92, 0x35, 0xb6, 0xc7, 0xeb, 0x51, 0xbc, 0x3f, 0xda, 0x1d, 0x83, 0x5d, 0x20, 0xe5, 0x11, + 0x78, 0x05, 0x3a, 0x5e, 0x04, 0x29, 0x1d, 0x29, 0x23, 0x8a, 0x08, 0x6f, 0x1a, 0xca, 0x3c, 0x02, + 0xda, 0x99, 0x35, 0x38, 0x49, 0x13, 0x1a, 0x2a, 0xcf, 0xbd, 0xe7, 0xdc, 0x7b, 0xce, 0x1d, 0xcf, + 0x5d, 0x58, 0x12, 0x93, 0x80, 0x45, 0x76, 0x10, 0xfa, 0xc2, 0x47, 0x79, 0x31, 0xa0, 0x9e, 0x1f, + 0xad, 0x6d, 0x38, 0x5c, 0x0c, 0x46, 0x1d, 0xbb, 0xeb, 0xbb, 0x75, 0xc7, 0x77, 0xfc, 0xba, 0x84, + 0x3b, 0xa3, 0xbe, 0x8c, 0x64, 0x20, 0x4f, 0xaa, 0x6c, 0xed, 0xfe, 0x3c, 0x3d, 0xa4, 0x7d, 0xea, + 0xd1, 0xba, 0xcb, 0x5d, 0x1e, 0xd6, 0x83, 0x23, 0x47, 0x9d, 0x82, 0x8e, 0xfa, 0x55, 0x15, 0xd6, + 0x37, 0x00, 0x73, 0x4f, 0x06, 0x23, 0xef, 0x08, 0xdd, 0x83, 0x7a, 0xe2, 0xa0, 0x02, 0xaa, 0xa0, + 0xf6, 0xdf, 0xe6, 0x1d, 0x5b, 0x39, 0xb0, 0x25, 0x68, 0x37, 0xbd, 0xae, 0xdf, 0xe3, 0x9e, 0x43, + 0x24, 0x07, 0xed, 0x41, 0xbd, 0x47, 0x05, 0xad, 0x64, 0xaa, 0xa0, 0xb6, 0xd0, 0x78, 0x7c, 0x72, + 0xbe, 0xae, 0x7d, 0x3f, 0x5f, 0xdf, 0xba, 0x8d, 0xba, 0xfd, 0xda, 0x8b, 0x68, 0x9f, 0x35, 0x26, + 0x82, 0x1d, 0x0c, 0x79, 0x97, 0x11, 0xd9, 0xc9, 0xda, 0x81, 0xc6, 0x4c, 0x03, 0x2d, 0xc2, 0xa2, + 0x54, 0x6d, 0xbf, 0xd9, 0x25, 0xa6, 0x86, 0x96, 0xe1, 0x92, 0x0a, 0x77, 0x78, 0x24, 0x7c, 0x27, + 0xa4, 0xae, 0x09, 0x50, 0x05, 0x96, 0x55, 0xf2, 0xe9, 0xd0, 0xa7, 0xe2, 0x0f, 0x92, 0xb1, 0x3e, + 0x03, 0x98, 0x3f, 0x60, 0x21, 0x67, 0x11, 0xea, 0xc3, 0xfc, 0x90, 0x76, 0xd8, 0x30, 0xaa, 0x80, + 0x6a, 0xb6, 0x56, 0xda, 0x5c, 0xb6, 0xbb, 0x7e, 0x28, 0xd8, 0x38, 0xe8, 0xd8, 0x2f, 0x92, 0xfc, + 0x1e, 0xe5, 0x61, 0xe3, 0x51, 0xea, 0xfe, 0xc1, 0xad, 0xdc, 0xcb, 0xba, 0xed, 0x1e, 0x0d, 0x04, + 0x0b, 0x49, 0xda, 0x1d, 0xd5, 0x61, 0xbe, 0x9b, 0x98, 0x89, 0x2a, 0x19, 0xa9, 0xf3, 0xff, 0xec, + 0xf2, 0xb6, 0x1d, 0x27, 0x94, 0x36, 0x1b, 0x7a, 0xa2, 0x42, 0x52, 0x9a, 0x35, 0x81, 0x4b, 0x07, + 0x22, 0x64, 0xd4, 0xe5, 0x9e, 0xf3, 0x6f, 0xbd, 0x5a, 0x1f, 0x61, 0xf9, 0x9a, 0x74, 0x83, 0x8a, + 0xee, 0x20, 0x99, 0x21, 0x92, 0x61, 0xaa, 0xbf, 0x32, 0x9b, 0xe1, 0x1a, 0x9b, 0xa4, 0x34, 0xb4, + 0x05, 0x57, 0x78, 0xd4, 0x66, 0x5e, 0xaf, 0xed, 0xf7, 0xdb, 0x2a, 0xd7, 0x8e, 0x24, 0x57, 0x3e, + 0x0b, 0x83, 0x2c, 0xf3, 0xa8, 0xe9, 0xf5, 0x76, 0xfb, 0xaa, 0x4e, 0xb5, 0xb1, 0xd8, 0xdc, 0xe4, + 0xf2, 0x66, 0x22, 0x74, 0x17, 0x2e, 0xa4, 0xe5, 0xdc, 0xeb, 0xb1, 0xb1, 0x7c, 0x80, 0x3a, 0x29, + 0xa9, 0xdc, 0xf3, 0x24, 0xf5, 0xf7, 0x17, 0xfc, 0x6c, 0x6e, 0x4a, 0x25, 0x73, 0xdb, 0x29, 0x15, + 0x7b, 0x36, 0xa5, 0x75, 0x0c, 0x60, 0xf1, 0xb7, 0x08, 0x5a, 0x85, 0x86, 0xcb, 0xbd, 0xb6, 0xe0, + 0xae, 0xda, 0x93, 0x2c, 0x29, 0xb8, 0xdc, 0x3b, 0xe4, 0x2e, 0x93, 0x10, 0x1d, 0x2b, 0x28, 0x93, + 0x42, 0x74, 0x2c, 0xa1, 0x75, 0x98, 0x0d, 0xe9, 0x87, 0x4a, 0xb6, 0x0a, 0x6a, 0xa5, 0xcd, 0xc5, + 0x2b, 0x8b, 0x45, 0x12, 0xa4, 0xa5, 0x1b, 0xba, 0x99, 0x6b, 0xe9, 0x46, 0xce, 0xcc, 0xb7, 0x74, + 0x23, 0x6f, 0x16, 0x5a, 0xba, 0x51, 0x30, 0x8d, 0x96, 0x6e, 0x18, 0x66, 0xd1, 0xfa, 0x0a, 0xe0, + 0x82, 0xfc, 0x2b, 0x5f, 0x26, 0x23, 0xb0, 0x10, 0x6d, 0x5c, 0xd9, 0xd4, 0xd5, 0x59, 0xc3, 0x79, + 0x8e, 0x7d, 0x38, 0x09, 0x58, 0xba, 0xac, 0x08, 0xea, 0x1e, 0x4d, 0x5d, 0x15, 0x89, 0x3c, 0xa3, + 0x32, 0xcc, 0xbd, 0xa7, 0xc3, 0x11, 0x93, 0xa6, 0x8a, 0x44, 0x05, 0xd6, 0x3b, 0xa8, 0x27, 0x75, + 0xc9, 0xc6, 0xcd, 0x37, 0x6b, 0x37, 0xf7, 0x4d, 0x0d, 0x95, 0xa1, 0x79, 0x25, 0xf9, 0xaa, 0xb9, + 0x6f, 0x82, 0x1b, 0x54, 0xd2, 0x34, 0x33, 0x37, 0xa9, 0xa4, 0x69, 0x66, 0x1b, 0xdb, 0x27, 0x53, + 0xac, 0x9d, 0x4e, 0xb1, 0x76, 0x36, 0xc5, 0xda, 0xe5, 0x14, 0x83, 0xe3, 0x18, 0x83, 0x2f, 0x31, + 0x06, 0x27, 0x31, 0x06, 0xa7, 0x31, 0x06, 0x3f, 0x62, 0x0c, 0x7e, 0xc6, 0x58, 0xbb, 0x8c, 0x31, + 0xf8, 0x74, 0x81, 0xb5, 0xd3, 0x0b, 0xac, 0x9d, 0x5d, 0x60, 0xed, 0x6d, 0x21, 0x12, 0x7e, 0xc8, + 0x82, 0x4e, 0x27, 0x2f, 0x3f, 0x5a, 0x0f, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x81, 0xc9, 0x56, + 0x35, 0x2c, 0x05, 0x00, 0x00, } func (x Chunk_Encoding) String() string { @@ -359,6 +516,128 @@ func (this *Series) Equal(that interface{}) bool { } return true } +func (this *StreamingSeries) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*StreamingSeries) + if !ok { + that2, ok := that.(StreamingSeries) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if !this.Labels[i].Equal(that1.Labels[i]) { + return false + } + } + return true +} +func (this *StreamingSeriesBatch) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*StreamingSeriesBatch) + if !ok { + that2, ok := that.(StreamingSeriesBatch) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Series) != len(that1.Series) { + return false + } + for i := range this.Series { + if !this.Series[i].Equal(that1.Series[i]) { + return false + } + } + if this.IsEndOfSeriesStream != that1.IsEndOfSeriesStream { + return false + } + return true +} +func (this *StreamingChunks) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*StreamingChunks) + if !ok { + that2, ok := that.(StreamingChunks) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.SeriesIndex != that1.SeriesIndex { + return false + } + if len(this.Chunks) != len(that1.Chunks) { + return false + } + for i := range this.Chunks { + if !this.Chunks[i].Equal(&that1.Chunks[i]) { + return false + } + } + return true +} +func (this *StreamingChunksBatch) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*StreamingChunksBatch) + if !ok { + that2, ok := that.(StreamingChunksBatch) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Series) != len(that1.Series) { + return false + } + for i := range this.Series { + if !this.Series[i].Equal(that1.Series[i]) { + return false + } + } + return true +} func (this *AggrChunk) Equal(that interface{}) bool { if that == nil { return this == nil @@ -447,6 +726,58 @@ func (this *Series) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *StreamingSeries) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&storepb.StreamingSeries{") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *StreamingSeriesBatch) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&storepb.StreamingSeriesBatch{") + if this.Series != nil { + s = append(s, "Series: "+fmt.Sprintf("%#v", this.Series)+",\n") + } + s = append(s, "IsEndOfSeriesStream: "+fmt.Sprintf("%#v", this.IsEndOfSeriesStream)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *StreamingChunks) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&storepb.StreamingChunks{") + s = append(s, "SeriesIndex: "+fmt.Sprintf("%#v", this.SeriesIndex)+",\n") + if this.Chunks != nil { + vs := make([]*AggrChunk, len(this.Chunks)) + for i := range vs { + vs[i] = &this.Chunks[i] + } + s = append(s, "Chunks: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *StreamingChunksBatch) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&storepb.StreamingChunksBatch{") + if this.Series != nil { + s = append(s, "Series: "+fmt.Sprintf("%#v", this.Series)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} func (this *AggrChunk) GoString() string { if this == nil { return "nil" @@ -570,7 +901,7 @@ func (m *Series) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *AggrChunk) Marshal() (dAtA []byte, err error) { +func (m *StreamingSeries) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -580,34 +911,197 @@ func (m *AggrChunk) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *AggrChunk) MarshalTo(dAtA []byte) (int, error) { +func (m *StreamingSeries) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *AggrChunk) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *StreamingSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.Raw != nil { - { - size, err := m.Raw.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintTypes(dAtA, i, uint64(size)) } - i -= size - i = encodeVarintTypes(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa } - i-- - dAtA[i] = 0x1a - } - if m.MaxTime != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.MaxTime)) - i-- - dAtA[i] = 0x10 } - if m.MinTime != 0 { + return len(dAtA) - i, nil +} + +func (m *StreamingSeriesBatch) 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 *StreamingSeriesBatch) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StreamingSeriesBatch) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.IsEndOfSeriesStream { + i-- + if m.IsEndOfSeriesStream { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if len(m.Series) > 0 { + for iNdEx := len(m.Series) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Series[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 *StreamingChunks) 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 *StreamingChunks) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StreamingChunks) 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 m.SeriesIndex != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.SeriesIndex)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *StreamingChunksBatch) 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 *StreamingChunksBatch) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StreamingChunksBatch) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Series) > 0 { + for iNdEx := len(m.Series) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Series[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 *AggrChunk) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AggrChunk) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *AggrChunk) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Raw != nil { + { + size, err := m.Raw.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.MaxTime != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.MaxTime)) + i-- + dAtA[i] = 0x10 + } + if m.MinTime != 0 { i = encodeVarintTypes(dAtA, i, uint64(m.MinTime)) i-- dAtA[i] = 0x8 @@ -703,6 +1197,72 @@ func (m *Series) Size() (n int) { return n } +func (m *StreamingSeries) 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)) + } + } + return n +} + +func (m *StreamingSeriesBatch) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Series) > 0 { + for _, e := range m.Series { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if m.IsEndOfSeriesStream { + n += 2 + } + return n +} + +func (m *StreamingChunks) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SeriesIndex != 0 { + n += 1 + sovTypes(uint64(m.SeriesIndex)) + } + if len(m.Chunks) > 0 { + for _, e := range m.Chunks { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func (m *StreamingChunksBatch) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Series) > 0 { + for _, e := range m.Series { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + func (m *AggrChunk) Size() (n int) { if m == nil { return 0 @@ -775,6 +1335,63 @@ func (this *Series) String() string { }, "") return s } +func (this *StreamingSeries) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&StreamingSeries{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `}`, + }, "") + return s +} +func (this *StreamingSeriesBatch) String() string { + if this == nil { + return "nil" + } + repeatedStringForSeries := "[]*StreamingSeries{" + for _, f := range this.Series { + repeatedStringForSeries += strings.Replace(f.String(), "StreamingSeries", "StreamingSeries", 1) + "," + } + repeatedStringForSeries += "}" + s := strings.Join([]string{`&StreamingSeriesBatch{`, + `Series:` + repeatedStringForSeries + `,`, + `IsEndOfSeriesStream:` + fmt.Sprintf("%v", this.IsEndOfSeriesStream) + `,`, + `}`, + }, "") + return s +} +func (this *StreamingChunks) String() string { + if this == nil { + return "nil" + } + repeatedStringForChunks := "[]AggrChunk{" + for _, f := range this.Chunks { + repeatedStringForChunks += strings.Replace(strings.Replace(f.String(), "AggrChunk", "AggrChunk", 1), `&`, ``, 1) + "," + } + repeatedStringForChunks += "}" + s := strings.Join([]string{`&StreamingChunks{`, + `SeriesIndex:` + fmt.Sprintf("%v", this.SeriesIndex) + `,`, + `Chunks:` + repeatedStringForChunks + `,`, + `}`, + }, "") + return s +} +func (this *StreamingChunksBatch) String() string { + if this == nil { + return "nil" + } + repeatedStringForSeries := "[]*StreamingChunks{" + for _, f := range this.Series { + repeatedStringForSeries += strings.Replace(f.String(), "StreamingChunks", "StreamingChunks", 1) + "," + } + repeatedStringForSeries += "}" + s := strings.Join([]string{`&StreamingChunksBatch{`, + `Series:` + repeatedStringForSeries + `,`, + `}`, + }, "") + return s +} func (this *AggrChunk) String() string { if this == nil { return "nil" @@ -1033,6 +1650,393 @@ func (m *Series) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamingSeries) 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: StreamingSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamingSeries: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, github_com_grafana_mimir_pkg_mimirpb.LabelAdapter{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StreamingSeriesBatch) 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: StreamingSeriesBatch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamingSeriesBatch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Series", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 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.Series = append(m.Series, &StreamingSeries{}) + if err := m.Series[len(m.Series)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsEndOfSeriesStream", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsEndOfSeriesStream = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StreamingChunks) 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: StreamingChunks: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamingChunks: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesIndex", wireType) + } + m.SeriesIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeriesIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Chunks = append(m.Chunks, AggrChunk{}) + if err := m.Chunks[len(m.Chunks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StreamingChunksBatch) 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: StreamingChunksBatch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamingChunksBatch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Series", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 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.Series = append(m.Series, &StreamingChunks{}) + if err := m.Series[len(m.Series)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *AggrChunk) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/storegateway/storepb/types.proto b/pkg/storegateway/storepb/types.proto index b9b2e4d5098..8d299b675da 100644 --- a/pkg/storegateway/storepb/types.proto +++ b/pkg/storegateway/storepb/types.proto @@ -37,6 +37,25 @@ message Series { repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; } +message StreamingSeries { + repeated cortexpb.LabelPair labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/grafana/mimir/pkg/mimirpb.LabelAdapter"]; +} + +message StreamingSeriesBatch { + repeated StreamingSeries series = 1; + bool is_end_of_series_stream = 2; +} + +message StreamingChunks { + uint64 series_index = 1; // Index into list of all series previously sent with SeriesResponse messages by this storegateway during this query response. + repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; +} + +message StreamingChunksBatch { + repeated StreamingChunks series = 1; +} + + message AggrChunk { int64 min_time = 1; int64 max_time = 2;