Skip to content

Commit 4bc95c0

Browse files
authored
feat(dataobj): Add methods for querying data objects metadata (#16190)
1 parent 5aa9e47 commit 4bc95c0

File tree

6 files changed

+833
-25
lines changed

6 files changed

+833
-25
lines changed

pkg/dataobj/builder.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,9 +141,7 @@ func NewBuilder(cfg BuilderConfig) (*Builder, error) {
141141
return nil, fmt.Errorf("failed to create LRU cache: %w", err)
142142
}
143143

144-
var (
145-
metrics = newMetrics()
146-
)
144+
metrics := newMetrics()
147145
metrics.ObserveConfig(cfg)
148146

149147
return &Builder{

pkg/dataobj/metastore/metastore.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -234,7 +234,9 @@ func listObjectsFromStores(ctx context.Context, bucket objstore.Bucket, storePat
234234
g.Go(func() error {
235235
var err error
236236
objects[i], err = listObjects(ctx, bucket, path, start, end)
237-
if err != nil {
237+
// If the metastore object is not found, it means it's outside of any existing window
238+
// and we can safely ignore it.
239+
if err != nil && !bucket.IsObjNotFoundErr(err) {
238240
return fmt.Errorf("listing objects from metastore %s: %w", path, err)
239241
}
240242
return nil
@@ -252,7 +254,7 @@ func listObjects(ctx context.Context, bucket objstore.Bucket, path string, start
252254
var buf bytes.Buffer
253255
objectReader, err := bucket.Get(ctx, path)
254256
if err != nil {
255-
return nil, fmt.Errorf("getting metastore object: %w", err)
257+
return nil, err
256258
}
257259
n, err := buf.ReadFrom(objectReader)
258260
if err != nil {

pkg/dataobj/querier/metadata.go

Lines changed: 298 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,298 @@
1+
package querier
2+
3+
import (
4+
"context"
5+
"fmt"
6+
"io"
7+
"sort"
8+
"sync"
9+
"time"
10+
11+
"github.com/prometheus/common/model"
12+
"github.com/prometheus/prometheus/model/labels"
13+
"golang.org/x/sync/errgroup"
14+
15+
"github.com/grafana/loki/v3/pkg/dataobj"
16+
"github.com/grafana/loki/v3/pkg/logproto"
17+
"github.com/grafana/loki/v3/pkg/logql"
18+
)
19+
20+
// SelectSeries implements querier.Store
21+
func (s *Store) SelectSeries(ctx context.Context, req logql.SelectLogParams) ([]logproto.SeriesIdentifier, error) {
22+
objects, err := s.objectsForTimeRange(ctx, req.Start, req.End)
23+
if err != nil {
24+
return nil, err
25+
}
26+
27+
shard, err := parseShards(req.Shards)
28+
if err != nil {
29+
return nil, err
30+
}
31+
32+
var matchers []*labels.Matcher
33+
if req.Selector != "" {
34+
expr, err := req.LogSelector()
35+
if err != nil {
36+
return nil, err
37+
}
38+
matchers = expr.Matchers()
39+
}
40+
41+
uniqueSeries := &sync.Map{}
42+
43+
processor := newStreamProcessor(req.Start, req.End, matchers, objects, shard)
44+
45+
err = processor.ProcessParallel(ctx, func(h uint64, stream dataobj.Stream) {
46+
uniqueSeries.Store(h, labelsToSeriesIdentifier(stream.Labels))
47+
})
48+
if err != nil {
49+
return nil, err
50+
}
51+
var result []logproto.SeriesIdentifier
52+
53+
// Convert sync.Map to slice
54+
uniqueSeries.Range(func(_, value interface{}) bool {
55+
if sid, ok := value.(logproto.SeriesIdentifier); ok {
56+
result = append(result, sid)
57+
}
58+
return true
59+
})
60+
61+
return result, nil
62+
}
63+
64+
// LabelNamesForMetricName implements querier.Store
65+
func (s *Store) LabelNamesForMetricName(ctx context.Context, _ string, from, through model.Time, _ string, matchers ...*labels.Matcher) ([]string, error) {
66+
start, end := from.Time(), through.Time()
67+
objects, err := s.objectsForTimeRange(ctx, start, end)
68+
if err != nil {
69+
return nil, err
70+
}
71+
72+
processor := newStreamProcessor(start, end, matchers, objects, noShard)
73+
uniqueNames := sync.Map{}
74+
75+
err = processor.ProcessParallel(ctx, func(_ uint64, stream dataobj.Stream) {
76+
for _, label := range stream.Labels {
77+
uniqueNames.Store(label.Name, nil)
78+
}
79+
})
80+
if err != nil {
81+
return nil, err
82+
}
83+
84+
names := []string{}
85+
uniqueNames.Range(func(key, _ interface{}) bool {
86+
names = append(names, key.(string))
87+
return true
88+
})
89+
90+
sort.Strings(names)
91+
92+
return names, nil
93+
}
94+
95+
// LabelValuesForMetricName implements querier.Store
96+
func (s *Store) LabelValuesForMetricName(ctx context.Context, _ string, from, through model.Time, _ string, labelName string, matchers ...*labels.Matcher) ([]string, error) {
97+
start, end := from.Time(), through.Time()
98+
99+
requireLabel, err := labels.NewMatcher(labels.MatchNotEqual, labelName, "")
100+
if err != nil {
101+
return nil, fmt.Errorf("failed to instantiate label matcher: %w", err)
102+
}
103+
104+
matchers = append(matchers, requireLabel)
105+
106+
objects, err := s.objectsForTimeRange(ctx, start, end)
107+
if err != nil {
108+
return nil, err
109+
}
110+
111+
processor := newStreamProcessor(start, end, matchers, objects, noShard)
112+
uniqueValues := sync.Map{}
113+
114+
err = processor.ProcessParallel(ctx, func(_ uint64, stream dataobj.Stream) {
115+
uniqueValues.Store(stream.Labels.Get(labelName), nil)
116+
})
117+
if err != nil {
118+
return nil, err
119+
}
120+
121+
values := []string{}
122+
uniqueValues.Range(func(key, _ interface{}) bool {
123+
values = append(values, key.(string))
124+
return true
125+
})
126+
127+
sort.Strings(values)
128+
129+
return values, nil
130+
}
131+
132+
var streamsPool = sync.Pool{
133+
New: func() any {
134+
streams := make([]dataobj.Stream, 1024)
135+
return &streams
136+
},
137+
}
138+
139+
// streamProcessor handles processing of unique series with custom collection logic
140+
type streamProcessor struct {
141+
predicate dataobj.StreamsPredicate
142+
seenSeries *sync.Map
143+
objects []*dataobj.Object
144+
shard logql.Shard
145+
}
146+
147+
// newStreamProcessor creates a new streamProcessor with the given parameters
148+
func newStreamProcessor(start, end time.Time, matchers []*labels.Matcher, objects []*dataobj.Object, shard logql.Shard) *streamProcessor {
149+
// Create a time range predicate
150+
var predicate dataobj.StreamsPredicate = dataobj.TimeRangePredicate[dataobj.StreamsPredicate]{
151+
StartTime: start,
152+
EndTime: end,
153+
IncludeStart: true,
154+
IncludeEnd: true,
155+
}
156+
157+
// If there are any matchers, combine them with an AND predicate
158+
if len(matchers) > 0 {
159+
predicate = dataobj.AndPredicate[dataobj.StreamsPredicate]{
160+
Left: predicate,
161+
Right: matchersToPredicate(matchers),
162+
}
163+
}
164+
165+
return &streamProcessor{
166+
predicate: predicate,
167+
seenSeries: &sync.Map{},
168+
objects: objects,
169+
shard: shard,
170+
}
171+
}
172+
173+
// matchersToPredicate converts a list of matchers to a dataobj.StreamsPredicate
174+
func matchersToPredicate(matchers []*labels.Matcher) dataobj.StreamsPredicate {
175+
var left dataobj.StreamsPredicate
176+
for _, matcher := range matchers {
177+
var right dataobj.StreamsPredicate
178+
switch matcher.Type {
179+
case labels.MatchEqual:
180+
right = dataobj.LabelMatcherPredicate{Name: matcher.Name, Value: matcher.Value}
181+
default:
182+
right = dataobj.LabelFilterPredicate{Name: matcher.Name, Keep: func(_, value string) bool {
183+
return matcher.Matches(value)
184+
}}
185+
}
186+
if left == nil {
187+
left = right
188+
} else {
189+
left = dataobj.AndPredicate[dataobj.StreamsPredicate]{
190+
Left: left,
191+
Right: right,
192+
}
193+
}
194+
}
195+
return left
196+
}
197+
198+
// ProcessParallel processes series from multiple readers in parallel
199+
func (sp *streamProcessor) ProcessParallel(ctx context.Context, onNewStream func(uint64, dataobj.Stream)) error {
200+
readers, err := shardStreamReaders(ctx, sp.objects, sp.shard)
201+
if err != nil {
202+
return err
203+
}
204+
205+
// set predicate on all readers
206+
for _, reader := range readers {
207+
if err := reader.SetPredicate(sp.predicate); err != nil {
208+
return err
209+
}
210+
}
211+
212+
g, ctx := errgroup.WithContext(ctx)
213+
for _, reader := range readers {
214+
g.Go(func() error {
215+
return sp.processSingleReader(ctx, reader, onNewStream)
216+
})
217+
}
218+
return g.Wait()
219+
}
220+
221+
func (sp *streamProcessor) processSingleReader(ctx context.Context, reader *dataobj.StreamsReader, onNewStream func(uint64, dataobj.Stream)) error {
222+
var (
223+
streamsPtr = streamsPool.Get().(*[]dataobj.Stream)
224+
streams = *streamsPtr
225+
buf = make([]byte, 0, 1024)
226+
h uint64
227+
)
228+
229+
defer streamsPool.Put(streamsPtr)
230+
231+
for {
232+
n, err := reader.Read(ctx, streams)
233+
if err != nil && err != io.EOF {
234+
return err
235+
}
236+
if n == 0 {
237+
break
238+
}
239+
for _, stream := range streams[:n] {
240+
h, buf = stream.Labels.HashWithoutLabels(buf, []string(nil)...)
241+
// Try to claim this hash first
242+
if _, seen := sp.seenSeries.LoadOrStore(h, nil); seen {
243+
continue
244+
}
245+
onNewStream(h, stream)
246+
}
247+
}
248+
return nil
249+
}
250+
251+
func labelsToSeriesIdentifier(labels labels.Labels) logproto.SeriesIdentifier {
252+
series := make([]logproto.SeriesIdentifier_LabelsEntry, len(labels))
253+
for i, label := range labels {
254+
series[i] = logproto.SeriesIdentifier_LabelsEntry{
255+
Key: label.Name,
256+
Value: label.Value,
257+
}
258+
}
259+
return logproto.SeriesIdentifier{
260+
Labels: series,
261+
}
262+
}
263+
264+
// shardStreamReaders fetches metadata of objects in parallel and shards them into a list of StreamsReaders
265+
func shardStreamReaders(ctx context.Context, objects []*dataobj.Object, shard logql.Shard) ([]*dataobj.StreamsReader, error) {
266+
// fetch all metadata of objects in parallel
267+
g, ctx := errgroup.WithContext(ctx)
268+
metadatas := make([]dataobj.Metadata, len(objects))
269+
for i, obj := range objects {
270+
g.Go(func() error {
271+
var err error
272+
metadatas[i], err = obj.Metadata(ctx)
273+
return err
274+
})
275+
}
276+
if err := g.Wait(); err != nil {
277+
return nil, err
278+
}
279+
// sectionIndex tracks the global section number across all objects to ensure consistent sharding
280+
var sectionIndex uint64
281+
var readers []*dataobj.StreamsReader
282+
for i, metadata := range metadatas {
283+
for j := 0; j < metadata.StreamsSections; j++ {
284+
// For sharded queries (e.g., "1 of 2"), we only read sections that belong to our shard
285+
// The section is assigned to a shard based on its global index across all objects
286+
if shard.PowerOfTwo != nil && shard.PowerOfTwo.Of > 1 {
287+
if sectionIndex%uint64(shard.PowerOfTwo.Of) != uint64(shard.PowerOfTwo.Shard) {
288+
sectionIndex++
289+
continue
290+
}
291+
}
292+
reader := dataobj.NewStreamsReader(objects[i], j)
293+
readers = append(readers, reader)
294+
sectionIndex++
295+
}
296+
}
297+
return readers, nil
298+
}

0 commit comments

Comments
 (0)