feat(dataobj-querier): Fixes streams section sharding & instrumentation (#16349)

pull/16356/head
Cyril Tovena 3 months ago committed by GitHub
parent 412f733301
commit 543ad8e894
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 5
      pkg/dataobj/querier/iter.go
  2. 80
      pkg/dataobj/querier/metadata.go
  3. 207
      pkg/dataobj/querier/store.go
  4. 272
      pkg/dataobj/querier/store_test.go
  5. 3
      pkg/loki/modules.go
  6. 28
      pkg/querier/queryrange/querysharding.go
  7. 63
      pkg/querier/queryrange/querysharding_test.go
  8. 161
      pkg/querier/store_combiner.go

@ -3,6 +3,7 @@ package querier
import (
"container/heap"
"context"
"fmt"
"io"
"sort"
"sync"
@ -74,7 +75,7 @@ func newEntryIterator(ctx context.Context,
for {
n, err := reader.Read(ctx, buf)
if err != nil && err != io.EOF {
return nil, err
return nil, fmt.Errorf("failed to read log records: %w", err)
}
if n == 0 && err == io.EOF {
@ -295,7 +296,7 @@ func newSampleIterator(ctx context.Context,
for {
n, err := reader.Read(ctx, buf)
if err != nil && err != io.EOF {
return nil, err
return nil, fmt.Errorf("failed to read log records: %w", err)
}
// Handle end of stream or empty read

@ -10,6 +10,7 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"go.uber.org/atomic"
@ -18,6 +19,7 @@ import (
"github.com/grafana/loki/v3/pkg/dataobj"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
util_log "github.com/grafana/loki/v3/pkg/util/log"
)
var streamsPool = sync.Pool{
@ -29,11 +31,17 @@ var streamsPool = sync.Pool{
// SelectSeries implements querier.Store
func (s *Store) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error) {
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End)
logger := util_log.WithContext(ctx, s.logger)
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End, logger)
if err != nil {
return nil, err
}
if len(objects) == 0 {
return nil, nil
}
shard, err := parseShards(req.Shards)
if err != nil {
return nil, err
@ -50,7 +58,7 @@ func (s *Store) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]
uniqueSeries := &sync.Map{}
processor := newStreamProcessor(req.Start, req.End, matchers, objects, shard, s.logger)
processor := newStreamProcessor(req.Start, req.End, matchers, objects, shard, logger)
err = processor.ProcessParallel(ctx, func(h uint64, stream dataobj.Stream) {
uniqueSeries.Store(h, labelsToSeriesIdentifier(stream.Labels))
@ -73,13 +81,18 @@ func (s *Store) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]
// LabelNamesForMetricName implements querier.Store
func (s *Store) LabelNamesForMetricName(ctx context.Context, _ string, from, through model.Time, _ string, matchers ...*labels.Matcher) ([]string, error) {
logger := util_log.WithContext(ctx, s.logger)
start, end := from.Time(), through.Time()
objects, err := s.objectsForTimeRange(ctx, start, end)
objects, err := s.objectsForTimeRange(ctx, start, end, logger)
if err != nil {
return nil, err
}
processor := newStreamProcessor(start, end, matchers, objects, noShard, s.logger)
if len(objects) == 0 {
return nil, nil
}
processor := newStreamProcessor(start, end, matchers, objects, noShard, logger)
uniqueNames := sync.Map{}
err = processor.ProcessParallel(ctx, func(_ uint64, stream dataobj.Stream) {
@ -104,6 +117,7 @@ func (s *Store) LabelNamesForMetricName(ctx context.Context, _ string, from, thr
// LabelValuesForMetricName implements querier.Store
func (s *Store) LabelValuesForMetricName(ctx context.Context, _ string, from, through model.Time, _ string, labelName string, matchers ...*labels.Matcher) ([]string, error) {
logger := util_log.WithContext(ctx, s.logger)
start, end := from.Time(), through.Time()
requireLabel, err := labels.NewMatcher(labels.MatchNotEqual, labelName, "")
@ -113,12 +127,16 @@ func (s *Store) LabelValuesForMetricName(ctx context.Context, _ string, from, th
matchers = append(matchers, requireLabel)
objects, err := s.objectsForTimeRange(ctx, start, end)
objects, err := s.objectsForTimeRange(ctx, start, end, logger)
if err != nil {
return nil, err
}
processor := newStreamProcessor(start, end, matchers, objects, noShard, s.logger)
if len(objects) == 0 {
return nil, nil
}
processor := newStreamProcessor(start, end, matchers, objects, noShard, logger)
uniqueValues := sync.Map{}
err = processor.ProcessParallel(ctx, func(_ uint64, stream dataobj.Stream) {
@ -143,13 +161,13 @@ func (s *Store) LabelValuesForMetricName(ctx context.Context, _ string, from, th
type streamProcessor struct {
predicate dataobj.StreamsPredicate
seenSeries *sync.Map
objects []*dataobj.Object
objects []object
shard logql.Shard
logger log.Logger
}
// newStreamProcessor creates a new streamProcessor with the given parameters
func newStreamProcessor(start, end time.Time, matchers []*labels.Matcher, objects []*dataobj.Object, shard logql.Shard, logger log.Logger) *streamProcessor {
func newStreamProcessor(start, end time.Time, matchers []*labels.Matcher, objects []object, shard logql.Shard, logger log.Logger) *streamProcessor {
return &streamProcessor{
predicate: streamPredicate(matchers, start, end),
seenSeries: &sync.Map{},
@ -172,6 +190,10 @@ func (sp *streamProcessor) ProcessParallel(ctx context.Context, onNewStream func
}()
start := time.Now()
span := opentracing.SpanFromContext(ctx)
if span != nil {
span.LogKV("msg", "processing streams", "total_readers", len(readers))
}
level.Debug(sp.logger).Log("msg", "processing streams", "total_readers", len(readers))
// set predicate on all readers
@ -185,6 +207,8 @@ func (sp *streamProcessor) ProcessParallel(ctx context.Context, onNewStream func
var processedStreams atomic.Int64
for _, reader := range readers {
g.Go(func() error {
span, ctx := opentracing.StartSpanFromContext(ctx, "streamProcessor.processSingleReader")
defer span.Finish()
n, err := sp.processSingleReader(ctx, reader, onNewStream)
if err != nil {
return err
@ -203,6 +227,9 @@ func (sp *streamProcessor) ProcessParallel(ctx context.Context, onNewStream func
"total_streams_processed", processedStreams.Load(),
"duration", time.Since(start),
)
if span != nil {
span.LogKV("msg", "streamProcessor.ProcessParallel done", "total_readers", len(readers), "total_streams_processed", processedStreams.Load(), "duration", time.Since(start))
}
return nil
}
@ -221,7 +248,7 @@ func (sp *streamProcessor) processSingleReader(ctx context.Context, reader *data
for {
n, err := reader.Read(ctx, streams)
if err != nil && err != io.EOF {
return processed, err
return processed, fmt.Errorf("failed to read streams: %w", err)
}
if n == 0 && err == io.EOF {
break
@ -253,29 +280,38 @@ func labelsToSeriesIdentifier(labels labels.Labels) logproto.SeriesIdentifier {
}
// shardStreamReaders fetches metadata of objects in parallel and shards them into a list of StreamsReaders
func shardStreamReaders(ctx context.Context, objects []*dataobj.Object, shard logql.Shard) ([]*dataobj.StreamsReader, error) {
func shardStreamReaders(ctx context.Context, objects []object, shard logql.Shard) ([]*dataobj.StreamsReader, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "shardStreamReaders")
defer span.Finish()
span.SetTag("objects", len(objects))
metadatas, err := fetchMetadatas(ctx, objects)
if err != nil {
return nil, err
}
// sectionIndex tracks the global section number across all objects to ensure consistent sharding
var sectionIndex uint64
var readers []*dataobj.StreamsReader
for i, metadata := range metadatas {
for j := 0; j < metadata.StreamsSections; j++ {
// For sharded queries (e.g., "1 of 2"), we only read sections that belong to our shard
// The section is assigned to a shard based on its global index across all objects
if shard.PowerOfTwo != nil && shard.PowerOfTwo.Of > 1 {
if sectionIndex%uint64(shard.PowerOfTwo.Of) != uint64(shard.PowerOfTwo.Shard) {
sectionIndex++
continue
}
if metadata.StreamsSections > 1 {
return nil, fmt.Errorf("unsupported multiple streams sections count: %d", metadata.StreamsSections)
}
// For sharded queries (e.g., "1 of 2"), we only read sections that belong to our shard
// The section is assigned to a shard based on its global index across all objects
if shard.PowerOfTwo != nil && shard.PowerOfTwo.Of > 1 {
if sectionIndex%uint64(shard.PowerOfTwo.Of) != uint64(shard.PowerOfTwo.Shard) {
sectionIndex++
continue
}
reader := streamReaderPool.Get().(*dataobj.StreamsReader)
reader.Reset(objects[i], j)
readers = append(readers, reader)
sectionIndex++
}
reader := streamReaderPool.Get().(*dataobj.StreamsReader)
reader.Reset(objects[i].Object, 0)
readers = append(readers, reader)
sectionIndex++
}
span.LogKV("msg", "shardStreamReaders done", "readers", len(readers))
return readers, nil
}

@ -6,12 +6,14 @@ import (
"fmt"
"io"
"slices"
"strings"
"sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/thanos-io/objstore"
@ -29,6 +31,7 @@ import (
storageconfig "github.com/grafana/loki/v3/pkg/storage/config"
"github.com/grafana/loki/v3/pkg/storage/stores/index/stats"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
util_log "github.com/grafana/loki/v3/pkg/util/log"
)
var (
@ -103,26 +106,41 @@ func NewStore(bucket objstore.Bucket, logger log.Logger) *Store {
}
}
func (s *Store) String() string {
return "dataobj"
}
// SelectLogs implements querier.Store
func (s *Store) SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error) {
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End)
logger := util_log.WithContext(ctx, s.logger)
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End, logger)
if err != nil {
return nil, err
}
if len(objects) == 0 {
return iter.NoopEntryIterator, nil
}
shard, err := parseShards(req.Shards)
if err != nil {
return nil, err
}
return selectLogs(ctx, objects, shard, req, s.logger)
return selectLogs(ctx, objects, shard, req, logger)
}
// SelectSamples implements querier.Store
func (s *Store) SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error) {
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End)
logger := util_log.WithContext(ctx, s.logger)
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End, logger)
if err != nil {
return nil, err
}
if len(objects) == 0 {
return iter.NoopSampleIterator, nil
}
shard, err := parseShards(req.Shards)
if err != nil {
@ -133,7 +151,7 @@ func (s *Store) SelectSamples(ctx context.Context, req logql.SelectSampleParams)
return nil, err
}
return selectSamples(ctx, objects, shard, expr, req.Start, req.End, s.logger)
return selectSamples(ctx, objects, shard, expr, req.Start, req.End, logger)
}
// Stats implements querier.Store
@ -154,8 +172,19 @@ func (s *Store) GetShards(_ context.Context, _ string, _ model.Time, _ model.Tim
return &logproto.ShardsResponse{}, nil
}
type object struct {
*dataobj.Object
path string
}
// objectsForTimeRange returns data objects for the given time range.
func (s *Store) objectsForTimeRange(ctx context.Context, from, through time.Time) ([]*dataobj.Object, error) {
func (s *Store) objectsForTimeRange(ctx context.Context, from, through time.Time, logger log.Logger) ([]object, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "objectsForTimeRange")
defer span.Finish()
span.SetTag("from", from)
span.SetTag("through", through)
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
@ -165,16 +194,28 @@ func (s *Store) objectsForTimeRange(ctx context.Context, from, through time.Time
return nil, err
}
level.Debug(s.logger).Log("msg", "found data objects for time range", "count", len(files), "from", from, "through", through)
logParams := []interface{}{
"msg", "found data objects for time range",
"count", len(files),
"from", from,
"through", through,
}
level.Debug(logger).Log(logParams...)
span.LogKV(logParams...)
span.LogKV("files", files)
objects := make([]*dataobj.Object, 0, len(files))
objects := make([]object, 0, len(files))
for _, path := range files {
objects = append(objects, dataobj.FromBucket(s.bucket, path))
objects = append(objects, object{
Object: dataobj.FromBucket(s.bucket, path),
path: path,
})
}
return objects, nil
}
func selectLogs(ctx context.Context, objects []*dataobj.Object, shard logql.Shard, req logql.SelectLogParams, logger log.Logger) (iter.EntryIterator, error) {
func selectLogs(ctx context.Context, objects []object, shard logql.Shard, req logql.SelectLogParams, logger log.Logger) (iter.EntryIterator, error) {
selector, err := req.LogSelector()
if err != nil {
return nil, err
@ -202,6 +243,11 @@ func selectLogs(ctx context.Context, objects []*dataobj.Object, shard logql.Shar
for i, obj := range shardedObjects {
g.Go(func() error {
span, ctx := opentracing.StartSpanFromContext(ctx, "object selectLogs")
defer span.Finish()
span.SetTag("object", obj.object.path)
span.SetTag("sections", len(obj.logReaders))
iterator, err := obj.selectLogs(ctx, streamsPredicate, logsPredicate, req)
if err != nil {
return err
@ -217,7 +263,7 @@ func selectLogs(ctx context.Context, objects []*dataobj.Object, shard logql.Shar
return iter.NewSortEntryIterator(iterators, req.Direction), nil
}
func selectSamples(ctx context.Context, objects []*dataobj.Object, shard logql.Shard, expr syntax.SampleExpr, start, end time.Time, logger log.Logger) (iter.SampleIterator, error) {
func selectSamples(ctx context.Context, objects []object, shard logql.Shard, expr syntax.SampleExpr, start, end time.Time, logger log.Logger) (iter.SampleIterator, error) {
shardedObjects, err := shardObjects(ctx, objects, shard, logger)
if err != nil {
return nil, err
@ -247,6 +293,11 @@ func selectSamples(ctx context.Context, objects []*dataobj.Object, shard logql.S
for i, obj := range shardedObjects {
g.Go(func() error {
span, ctx := opentracing.StartSpanFromContext(ctx, "object selectSamples")
defer span.Finish()
span.SetTag("object", obj.object.path)
span.SetTag("sections", len(obj.logReaders))
iterator, err := obj.selectSamples(ctx, streamsPredicate, logsPredicate, expr)
if err != nil {
return err
@ -263,6 +314,7 @@ func selectSamples(ctx context.Context, objects []*dataobj.Object, shard logql.S
}
type shardedObject struct {
object object
streamReader *dataobj.StreamsReader
logReaders []*dataobj.LogsReader
@ -270,33 +322,27 @@ type shardedObject struct {
streams map[int64]dataobj.Stream
}
func shardObjects(
ctx context.Context,
objects []*dataobj.Object,
shard logql.Shard,
logger log.Logger,
) ([]*shardedObject, error) {
metadatas, err := fetchMetadatas(ctx, objects)
if err != nil {
return nil, err
}
// shardSections returns a list of section indices to read per metadata based on the sharding configuration.
// The returned slice has the same length as the input metadatas, and each element contains the list of section indices
// that should be read for that metadata.
func shardSections(metadatas []dataobj.Metadata, shard logql.Shard) [][]int {
// Count total sections before sharding
var totalSections int
for _, metadata := range metadatas {
totalSections += metadata.LogsSections
if metadata.StreamsSections > 1 {
// We don't support multiple streams sections, but we still need to return a slice
// with the same length as the input metadatas.
return make([][]int, len(metadatas))
}
}
logger = log.With(logger, "objects", len(objects))
logger = log.With(logger, "total_sections", totalSections)
// sectionIndex tracks the global section number across all objects to ensure consistent sharding
var sectionIndex uint64
var shardedSections int
shardedReaders := make([]*shardedObject, 0, len(objects))
result := make([][]int, len(metadatas))
for i, metadata := range metadatas {
var reader *shardedObject
sections := make([]int, 0, metadata.LogsSections)
for j := 0; j < metadata.LogsSections; j++ {
if shard.PowerOfTwo != nil && shard.PowerOfTwo.Of > 1 {
if sectionIndex%uint64(shard.PowerOfTwo.Of) != uint64(shard.PowerOfTwo.Shard) {
@ -304,28 +350,78 @@ func shardObjects(
continue
}
}
shardedSections++
sections = append(sections, j)
sectionIndex++
}
result[i] = sections
}
if reader == nil {
reader = shardedObjectsPool.Get().(*shardedObject)
reader.streamReader = streamReaderPool.Get().(*dataobj.StreamsReader)
reader.streamReader.Reset(objects[i], j)
}
return result
}
func shardObjects(
ctx context.Context,
objects []object,
shard logql.Shard,
logger log.Logger,
) ([]*shardedObject, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "shardObjects")
defer span.Finish()
metadatas, err := fetchMetadatas(ctx, objects)
if err != nil {
return nil, err
}
// Get the sections to read per metadata
sectionsPerMetadata := shardSections(metadatas, shard)
// Count total sections that will be read
var totalSections int
var objectSections []int
for i, sections := range sectionsPerMetadata {
totalSections += len(sections)
objectSections = append(objectSections, metadatas[i].LogsSections)
}
shardedReaders := make([]*shardedObject, 0, len(objects))
for i, sections := range sectionsPerMetadata {
if len(sections) == 0 {
continue
}
reader := shardedObjectsPool.Get().(*shardedObject)
reader.streamReader = streamReaderPool.Get().(*dataobj.StreamsReader)
reader.object = objects[i]
reader.streamReader.Reset(objects[i].Object, 0)
for _, section := range sections {
logReader := logReaderPool.Get().(*dataobj.LogsReader)
logReader.Reset(objects[i], j)
logReader.Reset(objects[i].Object, section)
reader.logReaders = append(reader.logReaders, logReader)
sectionIndex++
}
// if reader is not nil, it means we have at least one log reader
if reader != nil {
shardedReaders = append(shardedReaders, reader)
}
shardedReaders = append(shardedReaders, reader)
}
var sectionsString strings.Builder
for _, sections := range sectionsPerMetadata {
sectionsString.WriteString(fmt.Sprintf("%v ", sections))
}
logParams := []interface{}{
"msg", "sharding sections",
"sharded_factor", shard.String(),
"total_objects", len(objects),
"total_sections", totalSections,
"object_sections", fmt.Sprintf("%v", objectSections),
"sharded_total_objects", len(shardedReaders),
"sharded_sections", sectionsString.String(),
}
level.Debug(logger).Log("msg", "sharding sections",
"sharded_sections", shardedSections,
"sharded_objects", len(shardedReaders),
"shard_factor", shard.PowerOfTwo.Of)
level.Debug(logger).Log(logParams...)
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV(logParams...)
}
return shardedReaders, nil
}
@ -339,6 +435,7 @@ func (s *shardedObject) reset() {
s.streamReader = nil
s.logReaders = s.logReaders[:0]
s.streamsIDs = s.streamsIDs[:0]
s.object = object{}
clear(s.streams)
}
@ -355,10 +452,15 @@ func (s *shardedObject) selectLogs(ctx context.Context, streamsPredicate dataobj
for i, reader := range s.logReaders {
g.Go(func() error {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("msg", "starting selectLogs in section", "index", i)
defer sp.LogKV("msg", "selectLogs section done", "index", i)
}
iter, err := newEntryIterator(ctx, s.streams, reader, req)
if err != nil {
return err
}
iterators[i] = iter
return nil
})
@ -383,6 +485,10 @@ func (s *shardedObject) selectSamples(ctx context.Context, streamsPredicate data
for i, reader := range s.logReaders {
g.Go(func() error {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("msg", "starting selectSamples in section", "index", i)
defer sp.LogKV("msg", "selectSamples section done", "index", i)
}
// extractor is not thread safe, so we need to create a new one for each object
extractor, err := expr.Extractor()
if err != nil {
@ -417,6 +523,10 @@ func (s *shardedObject) setPredicate(streamsPredicate dataobj.StreamsPredicate,
}
func (s *shardedObject) matchStreams(ctx context.Context) error {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("msg", "starting matchStreams")
defer sp.LogKV("msg", "matchStreams done")
}
streamsPtr := streamsPool.Get().(*[]dataobj.Stream)
defer streamsPool.Put(streamsPtr)
streams := *streamsPtr
@ -445,13 +555,17 @@ func (s *shardedObject) matchStreams(ctx context.Context) error {
}
// fetchMetadatas fetches metadata of objects in parallel
func fetchMetadatas(ctx context.Context, objects []*dataobj.Object) ([]dataobj.Metadata, error) {
func fetchMetadatas(ctx context.Context, objects []object) ([]dataobj.Metadata, error) {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("msg", "fetching metadata", "objects", len(objects))
defer sp.LogKV("msg", "fetched metadata")
}
g, ctx := errgroup.WithContext(ctx)
metadatas := make([]dataobj.Metadata, len(objects))
for i, obj := range objects {
g.Go(func() error {
var err error
metadatas[i], err = obj.Metadata(ctx)
metadatas[i], err = obj.Object.Metadata(ctx)
return err
})
}
@ -516,5 +630,8 @@ func parseShards(shards []string) (logql.Shard, error) {
if len(parsed) == 0 {
return noShard, nil
}
if parsed[0].Variant() != logql.PowerOfTwoVersion {
return noShard, fmt.Errorf("unsupported shard variant: %s", parsed[0].Variant())
}
return parsed[0], nil
}

@ -24,6 +24,7 @@ import (
"github.com/grafana/loki/v3/pkg/logql"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/querier/plan"
"github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index"
)
type sampleWithLabels struct {
@ -516,3 +517,274 @@ func readAllEntries(it iter.EntryIterator) ([]entryWithLabels, error) {
}
return result, it.Err()
}
func TestShardSections(t *testing.T) {
tests := []struct {
name string
metadatas []dataobj.Metadata
shard logql.Shard
want [][]int
}{
{
name: "single section, no sharding",
metadatas: []dataobj.Metadata{
{LogsSections: 1},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 1,
},
},
want: [][]int{
{0},
},
},
{
name: "multiple sections, no sharding",
metadatas: []dataobj.Metadata{
{LogsSections: 3},
{LogsSections: 2},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 1,
},
},
want: [][]int{
{0, 1, 2},
{0, 1},
},
},
{
name: "multiple sections, shard 0 of 2",
metadatas: []dataobj.Metadata{
{LogsSections: 3},
{LogsSections: 2},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 2,
},
},
want: [][]int{
{0, 2},
{1},
},
},
{
name: "multiple sections, shard 1 of 2",
metadatas: []dataobj.Metadata{
{LogsSections: 3},
{LogsSections: 2},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 1,
Of: 2,
},
},
want: [][]int{
{1},
{0},
},
},
{
name: "more sections than shards, shard 0 of 2",
metadatas: []dataobj.Metadata{
{LogsSections: 5},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 2,
},
},
want: [][]int{
{0, 2, 4},
},
},
{
name: "more sections than shards, shard 1 of 2",
metadatas: []dataobj.Metadata{
{LogsSections: 5},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 1,
Of: 2,
},
},
want: [][]int{
{1, 3},
},
},
{
name: "complex case, shard 0 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 7}, // sections 0,4
{LogsSections: 5}, // sections 0,4
{LogsSections: 3}, // sections 0
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 4,
},
},
want: [][]int{
{0, 4},
{1},
{0},
},
},
{
name: "complex case, shard 1 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 7}, // sections 1,5
{LogsSections: 5}, // sections 1
{LogsSections: 3}, // sections 1
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 1,
Of: 4,
},
},
want: [][]int{
{1, 5},
{2},
{1},
},
},
{
name: "complex case, shard 2 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 7}, // sections 2,6
{LogsSections: 5}, // sections 2
{LogsSections: 3}, // sections 2
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 2,
Of: 4,
},
},
want: [][]int{
{2, 6},
{3},
{2},
},
},
{
name: "complex case, shard 3 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 7}, // sections 3
{LogsSections: 5}, // sections 3
{LogsSections: 3}, // no sections
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 3,
Of: 4,
},
},
want: [][]int{
{3},
{0, 4},
{},
},
},
{
name: "less sections than shards, shard 1 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 1},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 1,
Of: 4,
},
},
want: [][]int{{}},
},
{
name: "less sections than shards, shard 0 of 4",
metadatas: []dataobj.Metadata{
{LogsSections: 1},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 4,
},
},
want: [][]int{{0}},
},
{
name: "multiple streams sections not supported",
metadatas: []dataobj.Metadata{
{LogsSections: 1, StreamsSections: 2},
},
shard: logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: 0,
Of: 1,
},
},
want: [][]int{nil},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := shardSections(tt.metadatas, tt.shard)
require.Equal(t, tt.want, got)
// For test cases with multiple shards, verify that all sections are covered exactly once
if tt.shard.PowerOfTwo != nil && tt.shard.PowerOfTwo.Of > 1 {
// Skip verification for invalid metadata
for _, meta := range tt.metadatas {
if meta.StreamsSections > 1 {
return
}
}
// Track which sections are assigned
type sectionKey struct {
metaIdx int
section int
}
sectionCounts := make(map[sectionKey]int)
// Count sections from all shards
for shardIdx := uint32(0); shardIdx < tt.shard.PowerOfTwo.Of; shardIdx++ {
shard := logql.Shard{
PowerOfTwo: &index.ShardAnnotation{
Shard: shardIdx,
Of: tt.shard.PowerOfTwo.Of,
},
}
sections := shardSections(tt.metadatas, shard)
for metaIdx, metaSections := range sections {
for _, section := range metaSections {
key := sectionKey{metaIdx: metaIdx, section: section}
sectionCounts[key]++
}
}
}
// Verify each section is assigned exactly once
for metaIdx, meta := range tt.metadatas {
for section := 0; section < meta.LogsSections; section++ {
key := sectionKey{metaIdx: metaIdx, section: section}
count := sectionCounts[key]
require.Equal(t, 1, count, "section %d in metadata %d was assigned %d times", section, metaIdx, count)
}
}
}
})
}
}

@ -1070,6 +1070,9 @@ func (t *Loki) initQueryFrontendMiddleware() (_ services.Service, err error) {
sort.Slice(schemas.Configs, func(i, j int) bool {
return schemas.Configs[i].From.UnixNano() < schemas.Configs[j].From.UnixNano()
})
for _, cfg := range schemas.Configs {
level.Debug(util_log.Logger).Log("msg", "schema config", "from", cfg.From, "row_shards", cfg.RowShards, "index_type", cfg.IndexType, "object_store", cfg.ObjectType, "schema", cfg.Schema)
}
}
middleware, stopper, err := queryrange.NewMiddleware(

@ -200,18 +200,24 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que
return ast.next.Do(ctx, r)
}
v := ast.limits.TSDBShardingStrategy(tenants[0])
version, err := logql.ParseShardVersion(v)
if err != nil {
level.Warn(spLogger).Log(
"msg", "failed to parse shard version",
"fallback", version.String(),
"err", err.Error(),
"user", tenants[0],
"query", r.GetQuery(),
)
var strategy logql.ShardingStrategy
if conf.IndexType == types.TSDBType {
v := ast.limits.TSDBShardingStrategy(tenants[0])
version, err := logql.ParseShardVersion(v)
if err != nil {
level.Warn(spLogger).Log(
"msg", "failed to parse shard version",
"fallback", version.String(),
"err", err.Error(),
"user", tenants[0],
"query", r.GetQuery(),
)
}
strategy = version.Strategy(resolver, uint64(ast.limits.TSDBMaxBytesPerShard(tenants[0])))
} else {
strategy = logql.NewPowerOfTwoStrategy(resolver)
}
strategy := version.Strategy(resolver, uint64(ast.limits.TSDBMaxBytesPerShard(tenants[0])))
// Merge global shard aggregations and tenant overrides.
limitShardAggregation := validation.IntersectionPerTenant(tenants, func(tenant string) []string {

@ -334,7 +334,6 @@ func Test_astMapper_QuerySizeLimits(t *testing.T) {
}
require.Equal(t, tc.expectedStatsHandlerHits, statsCalled)
})
}
}
@ -882,5 +881,67 @@ func Test_ASTMapper_MaxLookBackPeriod(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "foo")
_, err := mware.Do(ctx, lokiReq)
require.NoError(t, err)
}
func Test_ConstantShardingDefaultIndexType(t *testing.T) {
engineOpts := testEngineOpts
queryHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
req.(*LokiInstantRequest).Plan.AST = syntax.MustParseExpr(`{cluster="dev-us-central-0"}`)
shards, _, err := logql.ParseShards(req.(*LokiInstantRequest).Shards)
require.NoError(t, err)
require.Equal(t, 1, len(shards))
require.Equal(t, logql.PowerOfTwoVersion, shards[0].Variant())
require.Equal(t, uint32(32), shards[0].PowerOfTwo.Of)
return &LokiResponse{}, nil
})
statsHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
// This is the actual check that we're testing.
require.Equal(t, testTime.Add(-engineOpts.MaxLookBackPeriod).UnixMilli(), req.GetStart().UnixMilli())
return &IndexStatsResponse{
Response: &logproto.IndexStatsResponse{
Bytes: 1 << 10,
},
}, nil
})
mware := newASTMapperware(
ShardingConfigs{
{
From: config.DayTime{Time: model.Now().Add(-2 * 24 * time.Hour)},
RowShards: 2,
IndexType: "tsdb",
},
{
From: config.DayTime{Time: model.Now().Add(-1 * 24 * time.Hour)},
RowShards: 32,
},
},
engineOpts,
queryHandler,
queryHandler,
statsHandler,
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, tsdbMaxQueryParallelism: 1, queryTimeout: time.Second},
0,
[]string{},
)
q := `{cluster="dev-us-central-0"}`
lokiReq := &LokiInstantRequest{
Query: q,
Limit: 1000,
TimeTs: model.Now().Add(-1 * time.Hour).Time(),
Direction: logproto.FORWARD,
Path: "/loki/api/v1/query",
Plan: &plan.QueryPlan{
AST: syntax.MustParseExpr(q),
},
}
ctx := user.InjectOrgID(context.Background(), "foo")
_, err := mware.Do(ctx, lokiReq)
require.NoError(t, err)
}

@ -2,11 +2,16 @@ package querier
import (
"context"
"fmt"
"sort"
"strings"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/grafana/dskit/tenant"
"github.com/grafana/loki/v3/pkg/iter"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
@ -35,6 +40,12 @@ func NewStoreCombiner(stores []StoreConfig) *StoreCombiner {
sort.Slice(stores, func(i, j int) bool {
return stores[i].From < stores[j].From
})
for i, s := range stores {
stores[i] = StoreConfig{
Store: newInstrumentedStore(s.Store, i),
From: s.From,
}
}
return &StoreCombiner{stores: stores}
}
@ -342,3 +353,153 @@ func (sc *StoreCombiner) GetShards(ctx context.Context, userID string, from, thr
return groups[0], nil
}
}
type instrumentedStore struct {
Store Store
name string
}
func newInstrumentedStore(store Store, index int) *instrumentedStore {
storeName := fmt.Sprintf("#%d", index)
if stringer, ok := store.(interface{ String() string }); ok {
storeName = stringer.String()
}
return &instrumentedStore{
Store: store,
name: storeName,
}
}
func (s *instrumentedStore) SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".SelectSamples")
defer span.Finish()
tenantID, _ := tenant.TenantID(ctx)
span.SetTag("tenantID", tenantID)
span.SetTag("start", req.Start)
span.SetTag("end", req.End)
span.SetTag("shards", req.Shards)
if req.Plan != nil && req.Plan.AST != nil {
span.SetTag("expr", req.Plan.AST.String())
}
return s.Store.SelectSamples(ctx, req)
}
func (s *instrumentedStore) SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter.EntryIterator, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".SelectLogs")
defer span.Finish()
tenantID, _ := tenant.TenantID(ctx)
span.SetTag("tenantID", tenantID)
span.SetTag("start", req.Start)
span.SetTag("end", req.End)
span.SetTag("shards", req.Shards)
span.SetTag("direction", req.Direction)
if req.Plan != nil && req.Plan.AST != nil {
span.SetTag("expr", req.Plan.AST.String())
}
return s.Store.SelectLogs(ctx, req)
}
func (s *instrumentedStore) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".SelectSeries")
defer span.Finish()
tenantID, _ := tenant.TenantID(ctx)
span.SetTag("tenantID", tenantID)
span.SetTag("start", req.Start)
span.SetTag("end", req.End)
span.SetTag("shards", req.Shards)
if req.Plan != nil && req.Plan.AST != nil {
span.SetTag("expr", req.Plan.AST.String())
}
return s.Store.SelectSeries(ctx, req)
}
func (s *instrumentedStore) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".LabelValuesForMetricName")
defer span.Finish()
span.SetTag("tenantID", userID)
span.SetTag("from", from)
span.SetTag("through", through)
span.SetTag("metricName", metricName)
span.SetTag("labelName", labelName)
span.SetTag("matchers", stringifyMatchers(matchers))
return s.Store.LabelValuesForMetricName(ctx, userID, from, through, metricName, labelName, matchers...)
}
func (s *instrumentedStore) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".LabelNamesForMetricName")
defer span.Finish()
span.SetTag("tenantID", userID)
span.SetTag("from", from)
span.SetTag("through", through)
span.SetTag("metricName", metricName)
span.SetTag("matchers", stringifyMatchers(matchers))
return s.Store.LabelNamesForMetricName(ctx, userID, from, through, metricName, matchers...)
}
func (s *instrumentedStore) Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".Stats")
defer span.Finish()
span.SetTag("tenantID", userID)
span.SetTag("from", from)
span.SetTag("through", through)
span.SetTag("matchers", stringifyMatchers(matchers))
return s.Store.Stats(ctx, userID, from, through, matchers...)
}
func (s *instrumentedStore) Volume(ctx context.Context, userID string, from, through model.Time, limit int32, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".Volume")
defer span.Finish()
span.SetTag("tenantID", userID)
span.SetTag("from", from)
span.SetTag("through", through)
span.SetTag("limit", limit)
span.SetTag("targetLabels", targetLabels)
span.SetTag("aggregateBy", aggregateBy)
span.SetTag("matchers", stringifyMatchers(matchers))
return s.Store.Volume(ctx, userID, from, through, limit, targetLabels, aggregateBy, matchers...)
}
func (s *instrumentedStore) GetShards(ctx context.Context, userID string, from, through model.Time, targetBytesPerShard uint64, predicate chunk.Predicate) (*logproto.ShardsResponse, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "querier.Store."+s.name+".GetShards")
defer span.Finish()
span.SetTag("tenantID", userID)
span.SetTag("from", from)
span.SetTag("through", through)
span.SetTag("targetBytesPerShard", targetBytesPerShard)
span.SetTag("matchers", stringifyMatchers(predicate.Matchers))
return s.Store.GetShards(ctx, userID, from, through, targetBytesPerShard, predicate)
}
func (s *instrumentedStore) String() string {
return s.name
}
func stringifyMatchers(matchers []*labels.Matcher) string {
var result strings.Builder
for i, m := range matchers {
if i > 0 {
result.WriteString(", ")
}
result.WriteString(fmt.Sprintf("%s %s %s", m.Type.String(), m.Name, m.Value))
}
return result.String()
}

Loading…
Cancel
Save