github.com/yankunsam/loki/v2@v2.6.3-0.20220817130409-389df5235c27/pkg/storage/stores/series/series_index_store.go (about) 1 package series 2 3 import ( 4 "context" 5 "sort" 6 "sync" 7 8 "github.com/go-kit/log/level" 9 jsoniter "github.com/json-iterator/go" 10 "github.com/prometheus/client_golang/prometheus" 11 "github.com/prometheus/client_golang/prometheus/promauto" 12 "github.com/prometheus/common/model" 13 "github.com/prometheus/prometheus/model/labels" 14 15 "github.com/grafana/loki/pkg/logproto" 16 "github.com/grafana/loki/pkg/querier/astmapper" 17 "github.com/grafana/loki/pkg/storage/chunk" 18 "github.com/grafana/loki/pkg/storage/config" 19 storageerrors "github.com/grafana/loki/pkg/storage/errors" 20 "github.com/grafana/loki/pkg/storage/stores/index/stats" 21 "github.com/grafana/loki/pkg/storage/stores/series/index" 22 "github.com/grafana/loki/pkg/util" 23 "github.com/grafana/loki/pkg/util/extract" 24 util_log "github.com/grafana/loki/pkg/util/log" 25 "github.com/grafana/loki/pkg/util/spanlogger" 26 ) 27 28 var ( 29 indexLookupsPerQuery = promauto.NewHistogram(prometheus.HistogramOpts{ 30 Namespace: "loki", 31 Name: "chunk_store_index_lookups_per_query", 32 Help: "Distribution of #index lookups per query.", 33 Buckets: prometheus.ExponentialBuckets(1, 2, 5), 34 }) 35 preIntersectionPerQuery = promauto.NewHistogram(prometheus.HistogramOpts{ 36 Namespace: "loki", 37 Name: "chunk_store_series_pre_intersection_per_query", 38 Help: "Distribution of #series (pre intersection) per query.", 39 // A reasonable upper bound is around 100k - 10*(8^(6-1)) = 327k. 40 Buckets: prometheus.ExponentialBuckets(10, 8, 6), 41 }) 42 postIntersectionPerQuery = promauto.NewHistogram(prometheus.HistogramOpts{ 43 Namespace: "loki", 44 Name: "chunk_store_series_post_intersection_per_query", 45 Help: "Distribution of #series (post intersection) per query.", 46 // A reasonable upper bound is around 100k - 10*(8^(6-1)) = 327k. 47 Buckets: prometheus.ExponentialBuckets(10, 8, 6), 48 }) 49 chunksPerQuery = promauto.NewHistogram(prometheus.HistogramOpts{ 50 Namespace: "loki", 51 Name: "chunk_store_chunks_per_query", 52 Help: "Distribution of #chunks per query.", 53 // For 100k series for 7 week, could be 1.2m - 10*(8^(7-1)) = 2.6m. 54 Buckets: prometheus.ExponentialBuckets(10, 8, 7), 55 }) 56 ) 57 58 type chunkFetcher interface { 59 FetchChunks(ctx context.Context, chunks []chunk.Chunk, keys []string) ([]chunk.Chunk, error) 60 } 61 62 type IndexStore interface { 63 GetChunkRefs(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]logproto.ChunkRef, error) 64 GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) 65 LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) 66 LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error) 67 Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) 68 // SetChunkFilterer sets a chunk filter to be used when retrieving chunks. 69 // This is only used for GetSeries implementation. 70 // Todo we might want to pass it as a parameter to GetSeries instead. 71 SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) 72 } 73 74 type indexStore struct { 75 schema index.SeriesStoreSchema 76 index index.Client 77 schemaCfg config.SchemaConfig 78 fetcher chunkFetcher 79 chunkFilterer chunk.RequestChunkFilterer 80 chunkBatchSize int 81 } 82 83 func NewIndexStore(schemaCfg config.SchemaConfig, schema index.SeriesStoreSchema, index index.Client, fetcher chunkFetcher, chunkBatchSize int) IndexStore { 84 return &indexStore{ 85 schema: schema, 86 index: index, 87 schemaCfg: schemaCfg, 88 fetcher: fetcher, 89 chunkBatchSize: chunkBatchSize, 90 } 91 } 92 93 func (c *indexStore) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, allMatchers ...*labels.Matcher) ([]logproto.ChunkRef, error) { 94 log := util_log.WithContext(ctx, util_log.Logger) 95 // Check there is a metric name matcher of type equal, 96 metricNameMatcher, matchers, ok := extract.MetricNameMatcherFromMatchers(allMatchers) 97 if !ok || metricNameMatcher.Type != labels.MatchEqual { 98 return nil, storageerrors.ErrQueryMustContainMetricName 99 } 100 metricName := metricNameMatcher.Value 101 // Fetch the series IDs from the index, based on non-empty matchers from 102 // the query. 103 _, matchers = util.SplitFiltersAndMatchers(matchers) 104 seriesIDs, err := c.lookupSeriesByMetricNameMatchers(ctx, from, through, userID, metricName, matchers) 105 if err != nil { 106 return nil, err 107 } 108 level.Debug(log).Log("series-ids", len(seriesIDs)) 109 110 // Lookup the series in the index to get the chunks. 111 chunkIDs, err := c.lookupChunksBySeries(ctx, from, through, userID, seriesIDs) 112 if err != nil { 113 level.Error(log).Log("msg", "lookupChunksBySeries", "err", err) 114 return nil, err 115 } 116 level.Debug(log).Log("chunk-ids", len(chunkIDs)) 117 118 chunks, err := c.convertChunkIDsToChunkRefs(ctx, userID, chunkIDs) 119 if err != nil { 120 level.Error(log).Log("op", "convertChunkIDsToChunks", "err", err) 121 return nil, err 122 } 123 124 chunks = filterChunkRefsByTime(from, through, chunks) 125 level.Debug(log).Log("chunks-post-filtering", len(chunks)) 126 chunksPerQuery.Observe(float64(len(chunks))) 127 128 // We should return an empty chunks slice if there are no chunks. 129 if len(chunks) == 0 { 130 return []logproto.ChunkRef{}, nil 131 } 132 133 return chunks, nil 134 } 135 136 func (c *indexStore) SetChunkFilterer(f chunk.RequestChunkFilterer) { 137 c.chunkFilterer = f 138 } 139 140 type chunkGroup struct { 141 chunks []chunk.Chunk 142 keys []string 143 } 144 145 func (c chunkGroup) Len() int { return len(c.chunks) } 146 func (c chunkGroup) Swap(i, j int) { 147 c.chunks[i], c.chunks[j] = c.chunks[j], c.chunks[i] 148 c.keys[i], c.keys[j] = c.keys[j], c.keys[i] 149 } 150 func (c chunkGroup) Less(i, j int) bool { return c.keys[i] < c.keys[j] } 151 152 func (c *indexStore) GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) { 153 chks, err := c.GetChunkRefs(ctx, userID, from, through, matchers...) 154 if err != nil { 155 return nil, err 156 } 157 158 return c.chunksToSeries(ctx, chks, matchers) 159 } 160 161 func (c *indexStore) chunksToSeries(ctx context.Context, in []logproto.ChunkRef, matchers []*labels.Matcher) ([]labels.Labels, error) { 162 // download one per series and merge 163 // group chunks by series 164 chunksBySeries, keys := filterChunkRefsByUniqueFingerprint(c.schemaCfg, in) 165 166 results := make([]labels.Labels, 0, len(chunksBySeries)) 167 168 // bound concurrency 169 groups := make([]chunkGroup, 0, len(chunksBySeries)/c.chunkBatchSize+1) 170 171 split := c.chunkBatchSize 172 if len(chunksBySeries) < split { 173 split = len(chunksBySeries) 174 } 175 176 var chunkFilterer chunk.Filterer 177 if c.chunkFilterer != nil { 178 chunkFilterer = c.chunkFilterer.ForRequest(ctx) 179 } 180 181 for split > 0 { 182 groups = append(groups, chunkGroup{chunksBySeries[:split], keys[:split]}) 183 chunksBySeries = chunksBySeries[split:] 184 keys = keys[split:] 185 if len(chunksBySeries) < split { 186 split = len(chunksBySeries) 187 } 188 } 189 190 for _, group := range groups { 191 sort.Sort(group) 192 chunks, err := c.fetcher.FetchChunks(ctx, group.chunks, group.keys) 193 if err != nil { 194 return nil, err 195 } 196 197 outer: 198 for _, chk := range chunks { 199 for _, matcher := range matchers { 200 if matcher.Name == astmapper.ShardLabel || matcher.Name == labels.MetricName { 201 continue 202 } 203 if !matcher.Matches(chk.Metric.Get(matcher.Name)) { 204 continue outer 205 } 206 } 207 208 if chunkFilterer != nil && chunkFilterer.ShouldFilter(chk.Metric) { 209 continue outer 210 } 211 212 results = append(results, labels.NewBuilder(chk.Metric).Del(labels.MetricName).Labels()) 213 } 214 } 215 sort.Slice(results, func(i, j int) bool { 216 return labels.Compare(results[i], results[j]) < 0 217 }) 218 return results, nil 219 } 220 221 // LabelNamesForMetricName retrieves all label names for a metric name. 222 func (c *indexStore) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error) { 223 log, ctx := spanlogger.New(ctx, "SeriesStore.LabelNamesForMetricName") 224 defer log.Span.Finish() 225 226 // Fetch the series IDs from the index 227 seriesIDs, err := c.lookupSeriesByMetricNameMatchers(ctx, from, through, userID, metricName, nil) 228 if err != nil { 229 return nil, err 230 } 231 level.Debug(log).Log("series-ids", len(seriesIDs)) 232 233 // Lookup the series in the index to get label names. 234 labelNames, err := c.lookupLabelNamesBySeries(ctx, from, through, userID, seriesIDs) 235 if err != nil { 236 // looking up metrics by series is not supported falling back on chunks 237 if err == index.ErrNotSupported { 238 return c.lookupLabelNamesByChunks(ctx, from, through, userID, seriesIDs) 239 } 240 level.Error(log).Log("msg", "lookupLabelNamesBySeries", "err", err) 241 return nil, err 242 } 243 level.Debug(log).Log("labelNames", len(labelNames)) 244 245 return labelNames, nil 246 } 247 248 func (c *indexStore) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) { 249 log, ctx := spanlogger.New(ctx, "SeriesStore.LabelValuesForMetricName") 250 defer log.Span.Finish() 251 252 if len(matchers) != 0 { 253 return c.labelValuesForMetricNameWithMatchers(ctx, userID, from, through, metricName, labelName, matchers...) 254 } 255 256 level.Debug(log).Log("from", from, "through", through, "metricName", metricName, "labelName", labelName) 257 258 queries, err := c.schema.GetReadQueriesForMetricLabel(from, through, userID, metricName, labelName) 259 if err != nil { 260 return nil, err 261 } 262 263 entries, err := c.lookupEntriesByQueries(ctx, queries) 264 if err != nil { 265 return nil, err 266 } 267 // nolint:staticcheck 268 defer entriesPool.Put(entries) 269 270 var result util.UniqueStrings 271 for _, entry := range entries { 272 _, labelValue, err := index.ParseChunkTimeRangeValue(entry.RangeValue, entry.Value) 273 if err != nil { 274 return nil, err 275 } 276 result.Add(string(labelValue)) 277 } 278 return result.Strings(), nil 279 } 280 281 // LabelValuesForMetricName retrieves all label values for a single label name and metric name. 282 func (c *indexStore) labelValuesForMetricNameWithMatchers(ctx context.Context, userID string, from, through model.Time, metricName, labelName string, matchers ...*labels.Matcher) ([]string, error) { 283 // Otherwise get series which include other matchers 284 seriesIDs, err := c.lookupSeriesByMetricNameMatchers(ctx, from, through, userID, metricName, matchers) 285 if err != nil { 286 return nil, err 287 } 288 seriesIDsSet := make(map[string]struct{}, len(seriesIDs)) 289 for _, i := range seriesIDs { 290 seriesIDsSet[i] = struct{}{} 291 } 292 293 contains := func(id string) bool { 294 _, ok := seriesIDsSet[id] 295 return ok 296 } 297 298 // Fetch label values for label name that are part of the filtered chunks 299 queries, err := c.schema.GetReadQueriesForMetricLabel(from, through, userID, metricName, labelName) 300 if err != nil { 301 return nil, err 302 } 303 entries, err := c.lookupEntriesByQueries(ctx, queries) 304 if err != nil { 305 return nil, err 306 } 307 // nolint:staticcheck 308 defer entriesPool.Put(entries) 309 310 result := util.NewUniqueStrings(len(entries)) 311 for _, entry := range entries { 312 seriesID, labelValue, err := index.ParseChunkTimeRangeValue(entry.RangeValue, entry.Value) 313 if err != nil { 314 return nil, err 315 } 316 if contains(seriesID) { 317 result.Add(string(labelValue)) 318 } 319 } 320 321 return result.Strings(), nil 322 } 323 324 func (c *indexStore) lookupSeriesByMetricNameMatchers(ctx context.Context, from, through model.Time, userID, metricName string, matchers []*labels.Matcher) ([]string, error) { 325 // Check if one of the labels is a shard annotation, pass that information to lookupSeriesByMetricNameMatcher, 326 // and remove the label. 327 shard, shardLabelIndex, err := astmapper.ShardFromMatchers(matchers) 328 if err != nil { 329 return nil, err 330 } 331 332 if shard != nil { 333 matchers = append(matchers[:shardLabelIndex], matchers[shardLabelIndex+1:]...) 334 } 335 336 // Just get series for metric if there are no matchers 337 if len(matchers) == 0 { 338 indexLookupsPerQuery.Observe(1) 339 series, err := c.lookupSeriesByMetricNameMatcher(ctx, from, through, userID, metricName, nil, shard) 340 if err != nil { 341 preIntersectionPerQuery.Observe(float64(len(series))) 342 postIntersectionPerQuery.Observe(float64(len(series))) 343 } 344 return series, err 345 } 346 347 // Otherwise get series which include other matchers 348 incomingIDs := make(chan []string) 349 incomingErrors := make(chan error) 350 indexLookupsPerQuery.Observe(float64(len(matchers))) 351 for _, matcher := range matchers { 352 go func(matcher *labels.Matcher) { 353 ids, err := c.lookupSeriesByMetricNameMatcher(ctx, from, through, userID, metricName, matcher, shard) 354 if err != nil { 355 incomingErrors <- err 356 return 357 } 358 incomingIDs <- ids 359 }(matcher) 360 } 361 362 // Receive series IDs from all matchers, intersect as we go. 363 var ids []string 364 var preIntersectionCount int 365 var lastErr error 366 var cardinalityExceededErrors int 367 var cardinalityExceededError index.CardinalityExceededError 368 var initialized bool 369 for i := 0; i < len(matchers); i++ { 370 select { 371 case incoming := <-incomingIDs: 372 preIntersectionCount += len(incoming) 373 if !initialized { 374 ids = incoming 375 initialized = true 376 } else { 377 ids = intersectStrings(ids, incoming) 378 } 379 case err := <-incomingErrors: 380 // The idea is that if we have 2 matchers, and if one returns a lot of 381 // series and the other returns only 10 (a few), we don't lookup the first one at all. 382 // We just manually filter through the 10 series again using "filterChunksByMatchers", 383 // saving us from looking up and intersecting a lot of series. 384 if e, ok := err.(index.CardinalityExceededError); ok { 385 cardinalityExceededErrors++ 386 cardinalityExceededError = e 387 } else { 388 lastErr = err 389 } 390 } 391 } 392 393 // But if every single matcher returns a lot of series, then it makes sense to abort the query. 394 if cardinalityExceededErrors == len(matchers) { 395 return nil, cardinalityExceededError 396 } else if lastErr != nil { 397 return nil, lastErr 398 } 399 preIntersectionPerQuery.Observe(float64(preIntersectionCount)) 400 postIntersectionPerQuery.Observe(float64(len(ids))) 401 402 level.Debug(util_log.WithContext(ctx, util_log.Logger)). 403 Log("msg", "post intersection", "matchers", len(matchers), "ids", len(ids)) 404 return ids, nil 405 } 406 407 func (c *indexStore) lookupSeriesByMetricNameMatcher(ctx context.Context, from, through model.Time, userID, metricName string, matcher *labels.Matcher, shard *astmapper.ShardAnnotation) ([]string, error) { 408 return c.lookupIdsByMetricNameMatcher(ctx, from, through, userID, metricName, matcher, func(queries []index.Query) []index.Query { 409 return c.schema.FilterReadQueries(queries, shard) 410 }) 411 } 412 413 func (c *indexStore) lookupIdsByMetricNameMatcher(ctx context.Context, from, through model.Time, userID, metricName string, matcher *labels.Matcher, filter func([]index.Query) []index.Query) ([]string, error) { 414 var err error 415 var queries []index.Query 416 var labelName string 417 if matcher == nil { 418 queries, err = c.schema.GetReadQueriesForMetric(from, through, userID, metricName) 419 } else if matcher.Type == labels.MatchEqual { 420 labelName = matcher.Name 421 queries, err = c.schema.GetReadQueriesForMetricLabelValue(from, through, userID, metricName, matcher.Name, matcher.Value) 422 } else { 423 labelName = matcher.Name 424 queries, err = c.schema.GetReadQueriesForMetricLabel(from, through, userID, metricName, matcher.Name) 425 } 426 if err != nil { 427 return nil, err 428 } 429 430 if filter != nil { 431 queries = filter(queries) 432 } 433 434 entries, err := c.lookupEntriesByQueries(ctx, queries) 435 if e, ok := err.(index.CardinalityExceededError); ok { 436 e.MetricName = metricName 437 e.LabelName = labelName 438 return nil, e 439 } else if err != nil { 440 return nil, err 441 } 442 // nolint:staticcheck 443 defer entriesPool.Put(entries) 444 445 ids, err := parseIndexEntries(ctx, entries, matcher) 446 if err != nil { 447 return nil, err 448 } 449 450 return ids, nil 451 } 452 453 func parseIndexEntries(_ context.Context, entries []index.Entry, matcher *labels.Matcher) ([]string, error) { 454 // Nothing to do if there are no entries. 455 if len(entries) == 0 { 456 return nil, nil 457 } 458 459 matchSet := map[string]struct{}{} 460 if matcher != nil && matcher.Type == labels.MatchRegexp { 461 set := FindSetMatches(matcher.Value) 462 for _, v := range set { 463 matchSet[v] = struct{}{} 464 } 465 } 466 467 result := make([]string, 0, len(entries)) 468 for _, entry := range entries { 469 chunkKey, labelValue, err := index.ParseChunkTimeRangeValue(entry.RangeValue, entry.Value) 470 if err != nil { 471 return nil, err 472 } 473 474 // If the matcher is like a set (=~"a|b|c|d|...") and 475 // the label value is not in that set move on. 476 if len(matchSet) > 0 { 477 if _, ok := matchSet[string(labelValue)]; !ok { 478 continue 479 } 480 481 // If its in the set, then add it to set, we don't need to run 482 // matcher on it again. 483 result = append(result, chunkKey) 484 continue 485 } 486 487 if matcher != nil && !matcher.Matches(string(labelValue)) { 488 continue 489 } 490 result = append(result, chunkKey) 491 } 492 // Return ids sorted and deduped because they will be merged with other sets. 493 sort.Strings(result) 494 result = uniqueStrings(result) 495 return result, nil 496 } 497 498 var entriesPool = sync.Pool{ 499 New: func() interface{} { 500 return make([]index.Entry, 0, 1024) 501 }, 502 } 503 504 func (c *indexStore) lookupEntriesByQueries(ctx context.Context, queries []index.Query) ([]index.Entry, error) { 505 // Nothing to do if there are no queries. 506 if len(queries) == 0 { 507 return nil, nil 508 } 509 510 var lock sync.Mutex 511 entries := entriesPool.Get().([]index.Entry)[:0] 512 err := c.index.QueryPages(ctx, queries, func(query index.Query, resp index.ReadBatchResult) bool { 513 iter := resp.Iterator() 514 lock.Lock() 515 for iter.Next() { 516 entries = append(entries, index.Entry{ 517 TableName: query.TableName, 518 HashValue: query.HashValue, 519 RangeValue: iter.RangeValue(), 520 Value: iter.Value(), 521 }) 522 } 523 lock.Unlock() 524 return true 525 }) 526 if err != nil { 527 level.Error(util_log.WithContext(ctx, util_log.Logger)).Log("msg", "error querying storage", "err", err) 528 } 529 return entries, err 530 } 531 532 func (c *indexStore) lookupLabelNamesBySeries(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) { 533 log, ctx := spanlogger.New(ctx, "SeriesStore.lookupLabelNamesBySeries") 534 defer log.Span.Finish() 535 536 level.Debug(log).Log("seriesIDs", len(seriesIDs)) 537 queries := make([]index.Query, 0, len(seriesIDs)) 538 for _, seriesID := range seriesIDs { 539 qs, err := c.schema.GetLabelNamesForSeries(from, through, userID, []byte(seriesID)) 540 if err != nil { 541 return nil, err 542 } 543 queries = append(queries, qs...) 544 } 545 level.Debug(log).Log("queries", len(queries)) 546 entries, err := c.lookupEntriesByQueries(ctx, queries) 547 if err != nil { 548 return nil, err 549 } 550 // nolint:staticcheck 551 defer entriesPool.Put(entries) 552 553 level.Debug(log).Log("entries", len(entries)) 554 555 var result util.UniqueStrings 556 for _, entry := range entries { 557 lbs := []string{} 558 err := jsoniter.ConfigFastest.Unmarshal(entry.Value, &lbs) 559 if err != nil { 560 return nil, err 561 } 562 result.Add(lbs...) 563 } 564 return result.Strings(), nil 565 } 566 567 func (c *indexStore) lookupLabelNamesByChunks(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) { 568 log, ctx := spanlogger.New(ctx, "SeriesStore.lookupLabelNamesByChunks") 569 defer log.Span.Finish() 570 571 // Lookup the series in the index to get the chunks. 572 chunkIDs, err := c.lookupChunksBySeries(ctx, from, through, userID, seriesIDs) 573 if err != nil { 574 level.Error(log).Log("msg", "lookupChunksBySeries", "err", err) 575 return nil, err 576 } 577 level.Debug(log).Log("chunk-ids", len(chunkIDs)) 578 579 chunks, err := c.convertChunkIDsToChunks(ctx, userID, chunkIDs) 580 if err != nil { 581 level.Error(log).Log("err", "convertChunkIDsToChunks", "err", err) 582 return nil, err 583 } 584 585 // Filter out chunks that are not in the selected time range and keep a single chunk per fingerprint 586 filtered := filterChunksByTime(from, through, chunks) 587 filtered, keys := filterChunksByUniqueFingerprint(c.schemaCfg, filtered) 588 level.Debug(log).Log("Chunks post filtering", len(chunks)) 589 590 chunksPerQuery.Observe(float64(len(filtered))) 591 592 // Now fetch the actual chunk data from Memcache / S3 593 allChunks, err := c.fetcher.FetchChunks(ctx, filtered, keys) 594 if err != nil { 595 level.Error(log).Log("msg", "FetchChunks", "err", err) 596 return nil, err 597 } 598 return labelNamesFromChunks(allChunks), nil 599 } 600 601 func (c *indexStore) lookupChunksBySeries(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) { 602 queries := make([]index.Query, 0, len(seriesIDs)) 603 for _, seriesID := range seriesIDs { 604 qs, err := c.schema.GetChunksForSeries(from, through, userID, []byte(seriesID)) 605 if err != nil { 606 return nil, err 607 } 608 queries = append(queries, qs...) 609 } 610 611 entries, err := c.lookupEntriesByQueries(ctx, queries) 612 if err != nil { 613 return nil, err 614 } 615 // nolint:staticcheck 616 defer entriesPool.Put(entries) 617 618 result, err := parseIndexEntries(ctx, entries, nil) 619 return result, err 620 } 621 622 func (c *indexStore) convertChunkIDsToChunks(_ context.Context, userID string, chunkIDs []string) ([]chunk.Chunk, error) { 623 chunkSet := make([]chunk.Chunk, 0, len(chunkIDs)) 624 for _, chunkID := range chunkIDs { 625 chunk, err := chunk.ParseExternalKey(userID, chunkID) 626 if err != nil { 627 return nil, err 628 } 629 chunkSet = append(chunkSet, chunk) 630 } 631 632 return chunkSet, nil 633 } 634 635 func (c *indexStore) convertChunkIDsToChunkRefs(_ context.Context, userID string, chunkIDs []string) ([]logproto.ChunkRef, error) { 636 chunkSet := make([]logproto.ChunkRef, 0, len(chunkIDs)) 637 for _, chunkID := range chunkIDs { 638 chunk, err := chunk.ParseExternalKey(userID, chunkID) 639 if err != nil { 640 return nil, err 641 } 642 chunkSet = append(chunkSet, chunk.ChunkRef) 643 } 644 645 return chunkSet, nil 646 } 647 648 // old index stores do not implement stats -- skip 649 func (c *indexStore) Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) { 650 return nil, nil 651 }