-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
Copy pathquerier.go
1463 lines (1257 loc) · 43.1 KB
/
querier.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package querier
import (
"context"
"flag"
"net/http"
"sort"
"strconv"
"time"
"github.com/axiomhq/hyperloglog"
"github.com/dustin/go-humanize"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/google/uuid"
"github.com/grafana/dskit/httpgrpc"
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"golang.org/x/exp/slices"
"golang.org/x/sync/errgroup"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/v3/pkg/compactor/deletion"
"github.com/grafana/loki/v3/pkg/indexgateway"
"github.com/grafana/loki/v3/pkg/iter"
"github.com/grafana/loki/v3/pkg/loghttp"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
logql_log "github.com/grafana/loki/v3/pkg/logql/log"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/logqlmodel"
querier_limits "github.com/grafana/loki/v3/pkg/querier/limits"
"github.com/grafana/loki/v3/pkg/querier/plan"
"github.com/grafana/loki/v3/pkg/storage"
"github.com/grafana/loki/v3/pkg/storage/stores/index"
"github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume"
"github.com/grafana/loki/v3/pkg/storage/stores/index/stats"
listutil "github.com/grafana/loki/v3/pkg/util"
"github.com/grafana/loki/v3/pkg/util/httpreq"
"github.com/grafana/loki/v3/pkg/util/spanlogger"
util_validation "github.com/grafana/loki/v3/pkg/util/validation"
"github.com/grafana/loki/pkg/push"
)
const (
// How long the Tailer should wait - once there are no entries to read from ingesters -
// before checking if a new entry is available (to avoid spinning the CPU in a continuous
// check loop)
tailerWaitEntryThrottle = time.Second / 2
)
var nowFunc = func() time.Time { return time.Now() }
type interval struct {
start, end time.Time
}
// Config for a querier.
type Config struct {
TailMaxDuration time.Duration `yaml:"tail_max_duration"`
ExtraQueryDelay time.Duration `yaml:"extra_query_delay,omitempty"`
QueryIngestersWithin time.Duration `yaml:"query_ingesters_within,omitempty"`
IngesterQueryStoreMaxLookback time.Duration `yaml:"-"`
Engine logql.EngineOpts `yaml:"engine,omitempty"`
MaxConcurrent int `yaml:"max_concurrent"`
QueryStoreOnly bool `yaml:"query_store_only"`
QueryIngesterOnly bool `yaml:"query_ingester_only"`
MultiTenantQueriesEnabled bool `yaml:"multi_tenant_queries_enabled"`
PerRequestLimitsEnabled bool `yaml:"per_request_limits_enabled"`
}
// RegisterFlags register flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.Engine.RegisterFlagsWithPrefix("querier", f)
f.DurationVar(&cfg.TailMaxDuration, "querier.tail-max-duration", 1*time.Hour, "Maximum duration for which the live tailing requests are served.")
f.DurationVar(&cfg.ExtraQueryDelay, "querier.extra-query-delay", 0, "Time to wait before sending more than the minimum successful query requests.")
f.DurationVar(&cfg.QueryIngestersWithin, "querier.query-ingesters-within", 3*time.Hour, "Maximum lookback beyond which queries are not sent to ingester. 0 means all queries are sent to ingester.")
f.IntVar(&cfg.MaxConcurrent, "querier.max-concurrent", 4, "The maximum number of queries that can be simultaneously processed by the querier.")
f.BoolVar(&cfg.QueryStoreOnly, "querier.query-store-only", false, "Only query the store, and not attempt any ingesters. This is useful for running a standalone querier pool operating only against stored data.")
f.BoolVar(&cfg.QueryIngesterOnly, "querier.query-ingester-only", false, "When true, queriers only query the ingesters, and not stored data. This is useful when the object store is unavailable.")
f.BoolVar(&cfg.MultiTenantQueriesEnabled, "querier.multi-tenant-queries-enabled", false, "When true, allow queries to span multiple tenants.")
f.BoolVar(&cfg.PerRequestLimitsEnabled, "querier.per-request-limits-enabled", false, "When true, querier limits sent via a header are enforced.")
}
// Validate validates the config.
func (cfg *Config) Validate() error {
if cfg.QueryStoreOnly && cfg.QueryIngesterOnly {
return errors.New("querier.query_store_only and querier.query_ingester_only cannot both be true")
}
return nil
}
// Querier can select logs and samples and handle query requests.
type Querier interface {
logql.Querier
Label(ctx context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error)
Series(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error)
Tail(ctx context.Context, req *logproto.TailRequest, categorizedLabels bool) (*Tailer, error)
IndexStats(ctx context.Context, req *loghttp.RangeQuery) (*stats.Stats, error)
IndexShards(ctx context.Context, req *loghttp.RangeQuery, targetBytesPerShard uint64) (*logproto.ShardsResponse, error)
Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error)
DetectedFields(ctx context.Context, req *logproto.DetectedFieldsRequest) (*logproto.DetectedFieldsResponse, error)
Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error)
DetectedLabels(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.DetectedLabelsResponse, error)
WithPatternQuerier(patternQuerier PatterQuerier)
}
type Limits querier_limits.Limits
// Store is the store interface we need on the querier.
type Store interface {
storage.SelectStore
index.BaseReader
index.StatsReader
}
// SingleTenantQuerier handles single tenant queries.
type SingleTenantQuerier struct {
cfg Config
store Store
limits Limits
ingesterQuerier *IngesterQuerier
patternQuerier PatterQuerier
deleteGetter deleteGetter
metrics *Metrics
logger log.Logger
}
type deleteGetter interface {
GetAllDeleteRequestsForUser(ctx context.Context, userID string) ([]deletion.DeleteRequest, error)
}
// New makes a new Querier.
func New(cfg Config, store Store, ingesterQuerier *IngesterQuerier, limits Limits, d deleteGetter, r prometheus.Registerer, logger log.Logger) (*SingleTenantQuerier, error) {
return &SingleTenantQuerier{
cfg: cfg,
store: store,
ingesterQuerier: ingesterQuerier,
limits: limits,
deleteGetter: d,
metrics: NewMetrics(r),
logger: logger,
}, nil
}
// Select Implements logql.Querier which select logs via matchers and regex filters.
func (q *SingleTenantQuerier) SelectLogs(ctx context.Context, params logql.SelectLogParams) (iter.EntryIterator, error) {
var err error
params.Start, params.End, err = q.validateQueryRequest(ctx, params)
if err != nil {
return nil, err
}
params.QueryRequest.Deletes, err = q.deletesForUser(ctx, params.Start, params.End)
if err != nil {
level.Error(spanlogger.FromContext(ctx)).Log("msg", "failed loading deletes for user", "err", err)
}
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(params.Start, params.End)
sp := opentracing.SpanFromContext(ctx)
iters := []iter.EntryIterator{}
if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil {
// Make a copy of the request before modifying
// because the initial request is used below to query stores
queryRequestCopy := *params.QueryRequest
newParams := logql.SelectLogParams{
QueryRequest: &queryRequestCopy,
}
newParams.Start = ingesterQueryInterval.start
newParams.End = ingesterQueryInterval.end
if sp != nil {
sp.LogKV(
"msg", "querying ingester",
"params", newParams)
}
ingesterIters, err := q.ingesterQuerier.SelectLogs(ctx, newParams)
if err != nil {
return nil, err
}
iters = append(iters, ingesterIters...)
}
if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil {
params.Start = storeQueryInterval.start
params.End = storeQueryInterval.end
if sp != nil {
sp.LogKV(
"msg", "querying store",
"params", params)
}
storeIter, err := q.store.SelectLogs(ctx, params)
if err != nil {
return nil, err
}
iters = append(iters, storeIter)
}
if len(iters) == 1 {
return iters[0], nil
}
return iter.NewMergeEntryIterator(ctx, iters, params.Direction), nil
}
func (q *SingleTenantQuerier) SelectSamples(ctx context.Context, params logql.SelectSampleParams) (iter.SampleIterator, error) {
var err error
params.Start, params.End, err = q.validateQueryRequest(ctx, params)
if err != nil {
return nil, err
}
params.SampleQueryRequest.Deletes, err = q.deletesForUser(ctx, params.Start, params.End)
if err != nil {
level.Error(spanlogger.FromContext(ctx)).Log("msg", "failed loading deletes for user", "err", err)
}
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(params.Start, params.End)
iters := []iter.SampleIterator{}
if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil {
// Make a copy of the request before modifying
// because the initial request is used below to query stores
queryRequestCopy := *params.SampleQueryRequest
newParams := logql.SelectSampleParams{
SampleQueryRequest: &queryRequestCopy,
}
newParams.Start = ingesterQueryInterval.start
newParams.End = ingesterQueryInterval.end
ingesterIters, err := q.ingesterQuerier.SelectSample(ctx, newParams)
if err != nil {
return nil, err
}
iters = append(iters, ingesterIters...)
}
if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil {
params.Start = storeQueryInterval.start
params.End = storeQueryInterval.end
storeIter, err := q.store.SelectSamples(ctx, params)
if err != nil {
return nil, err
}
iters = append(iters, storeIter)
}
return iter.NewMergeSampleIterator(ctx, iters), nil
}
func (q *SingleTenantQuerier) deletesForUser(ctx context.Context, startT, endT time.Time) ([]*logproto.Delete, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
d, err := q.deleteGetter.GetAllDeleteRequestsForUser(ctx, userID)
if err != nil {
return nil, err
}
start := startT.UnixNano()
end := endT.UnixNano()
var deletes []*logproto.Delete
for _, del := range d {
if del.StartTime.UnixNano() <= end && del.EndTime.UnixNano() >= start {
deletes = append(deletes, &logproto.Delete{
Selector: del.Query,
Start: del.StartTime.UnixNano(),
End: del.EndTime.UnixNano(),
})
}
}
return deletes, nil
}
func (q *SingleTenantQuerier) isWithinIngesterMaxLookbackPeriod(maxLookback time.Duration, queryEnd time.Time) bool {
// if no lookback limits are configured, always consider this within the range of the lookback period
if maxLookback <= 0 {
return true
}
// find the first instance that we would want to query the ingester from...
ingesterOldestStartTime := time.Now().Add(-maxLookback)
// ...and if the query range ends before that, don't query the ingester
return queryEnd.After(ingesterOldestStartTime)
}
func (q *SingleTenantQuerier) calculateIngesterMaxLookbackPeriod() time.Duration {
mlb := time.Duration(-1)
if q.cfg.IngesterQueryStoreMaxLookback != 0 {
// IngesterQueryStoreMaxLookback takes the precedence over QueryIngestersWithin while also limiting the store query range.
mlb = q.cfg.IngesterQueryStoreMaxLookback
} else if q.cfg.QueryIngestersWithin != 0 {
mlb = q.cfg.QueryIngestersWithin
}
return mlb
}
func (q *SingleTenantQuerier) buildQueryIntervals(queryStart, queryEnd time.Time) (*interval, *interval) {
// limitQueryInterval is a flag for whether store queries should be limited to start time of ingester queries.
limitQueryInterval := false
// ingesterMLB having -1 means query ingester for whole duration.
if q.cfg.IngesterQueryStoreMaxLookback != 0 {
// IngesterQueryStoreMaxLookback takes the precedence over QueryIngestersWithin while also limiting the store query range.
limitQueryInterval = true
}
ingesterMLB := q.calculateIngesterMaxLookbackPeriod()
// query ingester for whole duration.
if ingesterMLB == -1 {
i := &interval{
start: queryStart,
end: queryEnd,
}
if limitQueryInterval {
// query only ingesters.
return i, nil
}
// query both stores and ingesters without limiting the query interval.
return i, i
}
ingesterQueryWithinRange := q.isWithinIngesterMaxLookbackPeriod(ingesterMLB, queryEnd)
// see if there is an overlap between ingester query interval and actual query interval, if not just do the store query.
if !ingesterQueryWithinRange {
return nil, &interval{
start: queryStart,
end: queryEnd,
}
}
ingesterOldestStartTime := time.Now().Add(-ingesterMLB)
// if there is an overlap and we are not limiting the query interval then do both store and ingester query for whole query interval.
if !limitQueryInterval {
i := &interval{
start: queryStart,
end: queryEnd,
}
return i, i
}
// since we are limiting the query interval, check if the query touches just the ingesters, if yes then query just the ingesters.
if ingesterOldestStartTime.Before(queryStart) {
return &interval{
start: queryStart,
end: queryEnd,
}, nil
}
// limit the start of ingester query interval to ingesterOldestStartTime.
ingesterQueryInterval := &interval{
start: ingesterOldestStartTime,
end: queryEnd,
}
// limit the end of ingester query interval to ingesterOldestStartTime.
storeQueryInterval := &interval{
start: queryStart,
end: ingesterOldestStartTime,
}
// query touches only ingester query interval so do not do store query.
if storeQueryInterval.start.After(storeQueryInterval.end) {
storeQueryInterval = nil
}
return ingesterQueryInterval, storeQueryInterval
}
// Label does the heavy lifting for a Label query.
func (q *SingleTenantQuerier) Label(ctx context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
if *req.Start, *req.End, err = validateQueryTimeRangeLimits(ctx, userID, q.limits, *req.Start, *req.End); err != nil {
return nil, err
}
var matchers []*labels.Matcher
if req.Query != "" {
matchers, err = syntax.ParseMatchers(req.Query, true)
if err != nil {
return nil, err
}
}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
g, ctx := errgroup.WithContext(ctx)
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(*req.Start, *req.End)
var ingesterValues [][]string
if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil {
g.Go(func() error {
var err error
timeFramedReq := *req
timeFramedReq.Start = &ingesterQueryInterval.start
timeFramedReq.End = &ingesterQueryInterval.end
ingesterValues, err = q.ingesterQuerier.Label(ctx, &timeFramedReq)
return err
})
}
var storeValues []string
if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil {
g.Go(func() error {
var (
err error
from = model.TimeFromUnixNano(storeQueryInterval.start.UnixNano())
through = model.TimeFromUnixNano(storeQueryInterval.end.UnixNano())
)
if req.Values {
storeValues, err = q.store.LabelValuesForMetricName(ctx, userID, from, through, "logs", req.Name, matchers...)
} else {
storeValues, err = q.store.LabelNamesForMetricName(ctx, userID, from, through, "logs", matchers...)
}
return err
})
}
if err := g.Wait(); err != nil {
return nil, err
}
results := append(ingesterValues, storeValues)
return &logproto.LabelResponse{
Values: listutil.MergeStringLists(results...),
}, nil
}
// Check implements the grpc healthcheck
func (*SingleTenantQuerier) Check(_ context.Context, _ *grpc_health_v1.HealthCheckRequest) (*grpc_health_v1.HealthCheckResponse, error) {
return &grpc_health_v1.HealthCheckResponse{Status: grpc_health_v1.HealthCheckResponse_SERVING}, nil
}
// Tail keeps getting matching logs from all ingesters for given query
func (q *SingleTenantQuerier) Tail(ctx context.Context, req *logproto.TailRequest, categorizedLabels bool) (*Tailer, error) {
err := q.checkTailRequestLimit(ctx)
if err != nil {
return nil, err
}
if req.Plan == nil {
parsed, err := syntax.ParseExpr(req.Query)
if err != nil {
return nil, err
}
req.Plan = &plan.QueryPlan{
AST: parsed,
}
}
deletes, err := q.deletesForUser(ctx, req.Start, time.Now())
if err != nil {
level.Error(spanlogger.FromContext(ctx)).Log("msg", "failed loading deletes for user", "err", err)
}
histReq := logql.SelectLogParams{
QueryRequest: &logproto.QueryRequest{
Selector: req.Query,
Start: req.Start,
End: time.Now(),
Limit: req.Limit,
Direction: logproto.BACKWARD,
Deletes: deletes,
Plan: req.Plan,
},
}
histReq.Start, histReq.End, err = q.validateQueryRequest(ctx, histReq)
if err != nil {
return nil, err
}
// Enforce the query timeout except when tailing, otherwise the tailing
// will be terminated once the query timeout is reached
tailCtx := ctx
tenantID, err := tenant.TenantID(tailCtx)
if err != nil {
return nil, errors.Wrap(err, "failed to load tenant")
}
queryTimeout := q.limits.QueryTimeout(tailCtx, tenantID)
queryCtx, cancelQuery := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancelQuery()
tailClients, err := q.ingesterQuerier.Tail(tailCtx, req)
if err != nil {
return nil, err
}
histIterators, err := q.SelectLogs(queryCtx, histReq)
if err != nil {
return nil, err
}
reversedIterator, err := iter.NewReversedIter(histIterators, req.Limit, true)
if err != nil {
return nil, err
}
return newTailer(
time.Duration(req.DelayFor)*time.Second,
tailClients,
reversedIterator,
func(connectedIngestersAddr []string) (map[string]logproto.Querier_TailClient, error) {
return q.ingesterQuerier.TailDisconnectedIngesters(tailCtx, req, connectedIngestersAddr)
},
q.cfg.TailMaxDuration,
tailerWaitEntryThrottle,
categorizedLabels,
q.metrics,
q.logger,
), nil
}
// Series fetches any matching series for a list of matcher sets
func (q *SingleTenantQuerier) Series(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
if req.Start, req.End, err = validateQueryTimeRangeLimits(ctx, userID, q.limits, req.Start, req.End); err != nil {
return nil, err
}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadlineCause(ctx, time.Now().Add(queryTimeout), errors.New("query timeout reached"))
defer cancel()
return q.awaitSeries(ctx, req)
}
func (q *SingleTenantQuerier) awaitSeries(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error) {
// buffer the channels to the # of calls they're expecting su
series := make(chan [][]logproto.SeriesIdentifier, 2)
errs := make(chan error, 2)
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(req.Start, req.End)
// fetch series from ingesters and store concurrently
if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil {
timeFramedReq := *req
timeFramedReq.Start = ingesterQueryInterval.start
timeFramedReq.End = ingesterQueryInterval.end
go func() {
// fetch series identifiers from ingesters
resps, err := q.ingesterQuerier.Series(ctx, &timeFramedReq)
if err != nil {
errs <- err
return
}
series <- resps
}()
} else {
// If only queriying the store or the query range does not overlap with the ingester max lookback period (defined by `query_ingesters_within`)
// then don't call out to the ingesters, and send an empty result back to the channel
series <- [][]logproto.SeriesIdentifier{}
}
if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil {
go func() {
storeValues, err := q.seriesForMatchers(ctx, storeQueryInterval.start, storeQueryInterval.end, req.GetGroups(), req.Shards)
if err != nil {
errs <- err
return
}
series <- [][]logproto.SeriesIdentifier{storeValues}
}()
} else {
// If we are not querying the store, send an empty result back to the channel
series <- [][]logproto.SeriesIdentifier{}
}
var sets [][]logproto.SeriesIdentifier
for i := 0; i < 2; i++ {
select {
case err := <-errs:
return nil, err
case s := <-series:
sets = append(sets, s...)
}
}
response := &logproto.SeriesResponse{
Series: make([]logproto.SeriesIdentifier, 0),
}
seen := make(map[uint64]struct{})
b := make([]byte, 0, 1024)
for _, set := range sets {
for _, s := range set {
key := s.Hash(b)
if _, exists := seen[key]; !exists {
seen[key] = struct{}{}
response.Series = append(response.Series, s)
}
}
}
return response, nil
}
// seriesForMatchers fetches series from the store for each matcher set
// TODO: make efficient if/when the index supports labels so we don't have to read chunks
func (q *SingleTenantQuerier) seriesForMatchers(
ctx context.Context,
from, through time.Time,
groups []string,
shards []string,
) ([]logproto.SeriesIdentifier, error) {
var results []logproto.SeriesIdentifier
// If no matchers were specified for the series query,
// we send a query with an empty matcher which will match every series.
if len(groups) == 0 {
var err error
results, err = q.seriesForMatcher(ctx, from, through, "", shards)
if err != nil {
return nil, err
}
} else {
for _, group := range groups {
ids, err := q.seriesForMatcher(ctx, from, through, group, shards)
if err != nil {
return nil, err
}
results = append(results, ids...)
}
}
return results, nil
}
// seriesForMatcher fetches series from the store for a given matcher
func (q *SingleTenantQuerier) seriesForMatcher(ctx context.Context, from, through time.Time, matcher string, shards []string) ([]logproto.SeriesIdentifier, error) {
var parsed syntax.Expr
var err error
if matcher != "" {
parsed, err = syntax.ParseExpr(matcher)
if err != nil {
return nil, err
}
}
ids, err := q.store.SelectSeries(ctx, logql.SelectLogParams{
QueryRequest: &logproto.QueryRequest{
Selector: matcher,
Limit: 1,
Start: from,
End: through,
Direction: logproto.FORWARD,
Shards: shards,
Plan: &plan.QueryPlan{
AST: parsed,
},
},
})
if err != nil {
return nil, err
}
return ids, nil
}
func (q *SingleTenantQuerier) validateQueryRequest(ctx context.Context, req logql.QueryParams) (time.Time, time.Time, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return time.Time{}, time.Time{}, err
}
selector, err := req.LogSelector()
if err != nil {
return time.Time{}, time.Time{}, err
}
matchers := selector.Matchers()
maxStreamMatchersPerQuery := q.limits.MaxStreamsMatchersPerQuery(ctx, userID)
if len(matchers) > maxStreamMatchersPerQuery {
return time.Time{}, time.Time{}, httpgrpc.Errorf(http.StatusBadRequest,
"max streams matchers per query exceeded, matchers-count > limit (%d > %d)", len(matchers), maxStreamMatchersPerQuery)
}
return validateQueryTimeRangeLimits(ctx, userID, q.limits, req.GetStart(), req.GetEnd())
}
type TimeRangeLimits querier_limits.TimeRangeLimits
func validateQueryTimeRangeLimits(ctx context.Context, userID string, limits TimeRangeLimits, from, through time.Time) (time.Time, time.Time, error) {
now := nowFunc()
// Clamp the time range based on the max query lookback.
maxQueryLookback := limits.MaxQueryLookback(ctx, userID)
if maxQueryLookback > 0 && from.Before(now.Add(-maxQueryLookback)) {
origStartTime := from
from = now.Add(-maxQueryLookback)
level.Debug(spanlogger.FromContext(ctx)).Log(
"msg", "the start time of the query has been manipulated because of the 'max query lookback' setting",
"original", origStartTime,
"updated", from)
}
maxQueryLength := limits.MaxQueryLength(ctx, userID)
if maxQueryLength > 0 && (through).Sub(from) > maxQueryLength {
return time.Time{}, time.Time{}, httpgrpc.Errorf(http.StatusBadRequest, util_validation.ErrQueryTooLong, (through).Sub(from), model.Duration(maxQueryLength))
}
if through.Before(from) {
return time.Time{}, time.Time{}, httpgrpc.Errorf(http.StatusBadRequest, util_validation.ErrQueryTooOld, model.Duration(maxQueryLookback))
}
return from, through, nil
}
func (q *SingleTenantQuerier) checkTailRequestLimit(ctx context.Context) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err
}
responses, err := q.ingesterQuerier.TailersCount(ctx)
// We are only checking active ingesters, and any error returned stops checking other ingesters
// so return that error here as well.
if err != nil {
return err
}
var maxCnt uint32
maxCnt = 0
for _, resp := range responses {
if resp > maxCnt {
maxCnt = resp
}
}
l := uint32(q.limits.MaxConcurrentTailRequests(ctx, userID))
if maxCnt >= l {
return httpgrpc.Errorf(http.StatusBadRequest,
"max concurrent tail requests limit exceeded, count > limit (%d > %d)", maxCnt+1, l)
}
return nil
}
func (q *SingleTenantQuerier) IndexStats(ctx context.Context, req *loghttp.RangeQuery) (*stats.Stats, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
start, end, err := validateQueryTimeRangeLimits(ctx, userID, q.limits, req.Start, req.End)
if err != nil {
return nil, err
}
matchers, err := syntax.ParseMatchers(req.Query, true)
if err != nil {
return nil, err
}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
return q.store.Stats(
ctx,
userID,
model.TimeFromUnixNano(start.UnixNano()),
model.TimeFromUnixNano(end.UnixNano()),
matchers...,
)
}
func (q *SingleTenantQuerier) IndexShards(
ctx context.Context,
req *loghttp.RangeQuery,
targetBytesPerShard uint64,
) (*logproto.ShardsResponse, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
start, end, err := validateQueryTimeRangeLimits(ctx, userID, q.limits, req.Start, req.End)
if err != nil {
return nil, err
}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
p, err := indexgateway.ExtractShardRequestMatchersAndAST(req.Query)
if err != nil {
return nil, err
}
shards, err := q.store.GetShards(
ctx,
userID,
model.TimeFromUnixNano(start.UnixNano()),
model.TimeFromUnixNano(end.UnixNano()),
targetBytesPerShard,
p,
)
if err != nil {
return nil, err
}
return shards, nil
}
func (q *SingleTenantQuerier) Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "Querier.Volume")
defer sp.Finish()
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
matchers, err := syntax.ParseMatchers(req.Matchers, true)
if err != nil && req.Matchers != seriesvolume.MatchAny {
return nil, err
}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
sp.LogKV(
"user", userID,
"from", req.From.Time(),
"through", req.Through.Time(),
"matchers", syntax.MatchersString(matchers),
"limit", req.Limit,
"targetLabels", req.TargetLabels,
"aggregateBy", req.AggregateBy,
)
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(req.From.Time(), req.Through.Time())
queryIngesters := !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil
queryStore := !q.cfg.QueryIngesterOnly && storeQueryInterval != nil
numResponses := 0
if queryIngesters {
numResponses++
}
if queryStore {
numResponses++
}
responses := make([]*logproto.VolumeResponse, 0, numResponses)
if queryIngesters {
// Make a copy of the request before modifying
// because the initial request is used below to query stores
resp, err := q.ingesterQuerier.Volume(
ctx,
userID,
model.TimeFromUnix(ingesterQueryInterval.start.Unix()),
model.TimeFromUnix(ingesterQueryInterval.end.Unix()),
req.Limit,
req.TargetLabels,
req.AggregateBy,
matchers...,
)
if err != nil {
return nil, err
}
responses = append(responses, resp)
}
if queryStore {
resp, err := q.store.Volume(
ctx,
userID,
model.TimeFromUnix(storeQueryInterval.start.Unix()),
model.TimeFromUnix(storeQueryInterval.end.Unix()),
req.Limit,
req.TargetLabels,
req.AggregateBy,
matchers...,
)
if err != nil {
return nil, err
}
responses = append(responses, resp)
}
return seriesvolume.Merge(responses, req.Limit), nil
}
// DetectedLabels fetches labels and values from store and ingesters and filters them by relevance criteria as per logs app.
func (q *SingleTenantQuerier) DetectedLabels(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.DetectedLabelsResponse, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
staticLabels := map[string]struct{}{"cluster": {}, "namespace": {}, "instance": {}, "pod": {}}
// Enforce the query timeout while querying backends
queryTimeout := q.limits.QueryTimeout(ctx, userID)
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout))
defer cancel()
g, ctx := errgroup.WithContext(ctx)
if req.Start, req.End, err = validateQueryTimeRangeLimits(ctx, userID, q.limits, req.Start, req.End); err != nil {
return nil, err
}
ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(req.Start, req.End)
// Fetch labels from ingesters
var ingesterLabels *logproto.LabelToValuesResponse
if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil {
g.Go(func() error {
var err error
splitReq := *req
splitReq.Start = ingesterQueryInterval.start
splitReq.End = ingesterQueryInterval.end
ingesterLabels, err = q.ingesterQuerier.DetectedLabel(ctx, &splitReq)
return err
})
}
// Fetch labels from the store
storeLabelsMap := make(map[string][]string)
if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil {
var matchers []*labels.Matcher
if req.Query != "" {
matchers, err = syntax.ParseMatchers(req.Query, true)
if err != nil {
return nil, err
}
}
g.Go(func() error {
var err error
start := model.TimeFromUnixNano(storeQueryInterval.start.UnixNano())
end := model.TimeFromUnixNano(storeQueryInterval.end.UnixNano())
storeLabels, err := q.store.LabelNamesForMetricName(ctx, userID, start, end, "logs", matchers...)
for _, label := range storeLabels {
values, err := q.store.LabelValuesForMetricName(ctx, userID, start, end, "logs", label, matchers...)
if err != nil {
return err
}
storeLabelsMap[label] = values
}
return err
})
}
if err := g.Wait(); err != nil {
return nil, err
}
if ingesterLabels == nil && len(storeLabelsMap) == 0 {
return &logproto.DetectedLabelsResponse{
DetectedLabels: []*logproto.DetectedLabel{},
}, nil
}
return &logproto.DetectedLabelsResponse{
DetectedLabels: countLabelsAndCardinality(storeLabelsMap, ingesterLabels, staticLabels),
}, nil
}
func countLabelsAndCardinality(storeLabelsMap map[string][]string, ingesterLabels *logproto.LabelToValuesResponse, staticLabels map[string]struct{}) []*logproto.DetectedLabel {
dlMap := make(map[string]*parsedFields)
if ingesterLabels != nil {
for label, val := range ingesterLabels.Labels {
if _, isStatic := staticLabels[label]; (isStatic && val.Values != nil) || !containsAllIDTypes(val.Values) {
_, ok := dlMap[label]