-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
distributor.go
1220 lines (1052 loc) · 40 KB
/
distributor.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 distributor
import (
"bytes"
"context"
"flag"
"fmt"
"math"
"net/http"
"sort"
"strconv"
"strings"
"sync"
"time"
"unicode"
"unsafe"
"github.com/buger/jsonparser"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/gogo/status"
"github.com/prometheus/prometheus/model/labels"
"github.com/twmb/franz-go/pkg/kgo"
"go.opentelemetry.io/collector/pdata/plog"
"google.golang.org/grpc/codes"
"github.com/grafana/dskit/httpgrpc"
"github.com/grafana/dskit/kv"
"github.com/grafana/dskit/limiter"
"github.com/grafana/dskit/ring"
ring_client "github.com/grafana/dskit/ring/client"
"github.com/grafana/dskit/services"
"github.com/grafana/dskit/tenant"
"github.com/grafana/dskit/user"
lru "github.com/hashicorp/golang-lru"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"go.uber.org/atomic"
"github.com/grafana/loki/v3/pkg/analytics"
"github.com/grafana/loki/v3/pkg/compactor/retention"
"github.com/grafana/loki/v3/pkg/distributor/clientpool"
"github.com/grafana/loki/v3/pkg/distributor/shardstreams"
"github.com/grafana/loki/v3/pkg/distributor/writefailures"
"github.com/grafana/loki/v3/pkg/ingester"
"github.com/grafana/loki/v3/pkg/ingester/client"
"github.com/grafana/loki/v3/pkg/kafka"
"github.com/grafana/loki/v3/pkg/loghttp/push"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql/log/logfmt"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/runtime"
"github.com/grafana/loki/v3/pkg/util"
"github.com/grafana/loki/v3/pkg/util/constants"
util_log "github.com/grafana/loki/v3/pkg/util/log"
lokiring "github.com/grafana/loki/v3/pkg/util/ring"
"github.com/grafana/loki/v3/pkg/validation"
)
const (
ringKey = "distributor"
ringAutoForgetUnhealthyPeriods = 2
)
var (
maxLabelCacheSize = 100000
rfStats = analytics.NewInt("distributor_replication_factor")
)
var allowedLabelsForLevel = map[string]struct{}{
"level": {}, "LEVEL": {}, "Level": {},
"severity": {}, "SEVERITY": {}, "Severity": {},
"lvl": {}, "LVL": {}, "Lvl": {},
}
// Config for a Distributor.
type Config struct {
// Distributors ring
DistributorRing RingConfig `yaml:"ring,omitempty"`
PushWorkerCount int `yaml:"push_worker_count"`
// For testing.
factory ring_client.PoolFactory `yaml:"-"`
// RateStore customizes the rate storing used by stream sharding.
RateStore RateStoreConfig `yaml:"rate_store"`
// WriteFailuresLoggingCfg customizes write failures logging behavior.
WriteFailuresLogging writefailures.Cfg `yaml:"write_failures_logging" doc:"description=Customize the logging of write failures."`
OTLPConfig push.GlobalOTLPConfig `yaml:"otlp_config"`
KafkaEnabled bool `yaml:"kafka_writes_enabled"`
IngesterEnabled bool `yaml:"ingester_writes_enabled"`
KafkaConfig kafka.Config `yaml:"-"`
}
// RegisterFlags registers distributor-related flags.
func (cfg *Config) RegisterFlags(fs *flag.FlagSet) {
cfg.OTLPConfig.RegisterFlags(fs)
cfg.DistributorRing.RegisterFlags(fs)
cfg.RateStore.RegisterFlagsWithPrefix("distributor.rate-store", fs)
cfg.WriteFailuresLogging.RegisterFlagsWithPrefix("distributor.write-failures-logging", fs)
fs.IntVar(&cfg.PushWorkerCount, "distributor.push-worker-count", 256, "Number of workers to push batches to ingesters.")
fs.BoolVar(&cfg.KafkaEnabled, "distributor.kafka-writes-enabled", false, "Enable writes to Kafka during Push requests.")
fs.BoolVar(&cfg.IngesterEnabled, "distributor.ingester-writes-enabled", true, "Enable writes to Ingesters during Push requests. Defaults to true.")
}
func (cfg *Config) Validate() error {
if !cfg.KafkaEnabled && !cfg.IngesterEnabled {
return fmt.Errorf("at least one of kafka and ingestor writes must be enabled")
}
return nil
}
// RateStore manages the ingestion rate of streams, populated by data fetched from ingesters.
type RateStore interface {
RateFor(tenantID string, streamHash uint64) (int64, float64)
}
type KafkaProducer interface {
ProduceSync(ctx context.Context, records []*kgo.Record) kgo.ProduceResults
Close()
}
// Distributor coordinates replicates and distribution of log streams.
type Distributor struct {
services.Service
cfg Config
logger log.Logger
clientCfg client.Config
tenantConfigs *runtime.TenantConfigs
tenantsRetention *retention.TenantsRetention
ingestersRing ring.ReadRing
validator *Validator
pool *ring_client.Pool
tee Tee
rateStore RateStore
shardTracker *ShardTracker
// The global rate limiter requires a distributors ring to count
// the number of healthy instances.
distributorsLifecycler *ring.BasicLifecycler
distributorsRing *ring.Ring
healthyInstancesCount *atomic.Uint32
rateLimitStrat string
subservices *services.Manager
subservicesWatcher *services.FailureWatcher
// Per-user rate limiter.
ingestionRateLimiter *limiter.RateLimiter
labelCache *lru.Cache
// Push failures rate limiter.
writeFailuresManager *writefailures.Manager
RequestParserWrapper push.RequestParserWrapper
// metrics
ingesterAppends *prometheus.CounterVec
ingesterAppendTimeouts *prometheus.CounterVec
replicationFactor prometheus.Gauge
streamShardCount prometheus.Counter
usageTracker push.UsageTracker
ingesterTasks chan pushIngesterTask
ingesterTaskWg sync.WaitGroup
// kafka
kafkaWriter KafkaProducer
partitionRing ring.PartitionRingReader
// kafka metrics
kafkaAppends *prometheus.CounterVec
kafkaWriteBytesTotal prometheus.Counter
kafkaWriteLatency prometheus.Histogram
kafkaRecordsPerRequest prometheus.Histogram
}
// New a distributor creates.
func New(
cfg Config,
clientCfg client.Config,
configs *runtime.TenantConfigs,
ingestersRing ring.ReadRing,
partitionRing ring.PartitionRingReader,
overrides Limits,
registerer prometheus.Registerer,
metricsNamespace string,
tee Tee,
usageTracker push.UsageTracker,
logger log.Logger,
) (*Distributor, error) {
factory := cfg.factory
if factory == nil {
factory = ring_client.PoolAddrFunc(func(addr string) (ring_client.PoolClient, error) {
return client.New(clientCfg, addr)
})
}
internalFactory := func(addr string) (ring_client.PoolClient, error) {
internalCfg := clientCfg
internalCfg.Internal = true
return client.New(internalCfg, addr)
}
validator, err := NewValidator(overrides, usageTracker)
if err != nil {
return nil, err
}
// Create the configured ingestion rate limit strategy (local or global).
var ingestionRateStrategy limiter.RateLimiterStrategy
var distributorsLifecycler *ring.BasicLifecycler
var distributorsRing *ring.Ring
var servs []services.Service
rateLimitStrat := validation.LocalIngestionRateStrategy
labelCache, err := lru.New(maxLabelCacheSize)
if err != nil {
return nil, err
}
if partitionRing == nil && cfg.KafkaEnabled {
return nil, fmt.Errorf("partition ring is required for kafka writes")
}
var kafkaWriter KafkaProducer
if cfg.KafkaEnabled {
kafkaClient, err := kafka.NewWriterClient(cfg.KafkaConfig, 20, logger, registerer)
if err != nil {
return nil, fmt.Errorf("failed to start kafka client: %w", err)
}
kafkaWriter = kafka.NewProducer(kafkaClient, cfg.KafkaConfig.ProducerMaxBufferedBytes,
prometheus.WrapRegistererWithPrefix("_kafka_", registerer))
}
d := &Distributor{
cfg: cfg,
logger: logger,
clientCfg: clientCfg,
tenantConfigs: configs,
tenantsRetention: retention.NewTenantsRetention(overrides),
ingestersRing: ingestersRing,
validator: validator,
pool: clientpool.NewPool("ingester", clientCfg.PoolConfig, ingestersRing, factory, logger, metricsNamespace),
labelCache: labelCache,
shardTracker: NewShardTracker(),
healthyInstancesCount: atomic.NewUint32(0),
rateLimitStrat: rateLimitStrat,
tee: tee,
usageTracker: usageTracker,
ingesterTasks: make(chan pushIngesterTask),
ingesterAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
Namespace: constants.Loki,
Name: "distributor_ingester_appends_total",
Help: "The total number of batch appends sent to ingesters.",
}, []string{"ingester"}),
ingesterAppendTimeouts: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
Namespace: constants.Loki,
Name: "distributor_ingester_append_timeouts_total",
Help: "The total number of failed batch appends sent to ingesters due to timeouts.",
}, []string{"ingester"}),
replicationFactor: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Namespace: constants.Loki,
Name: "distributor_replication_factor",
Help: "The configured replication factor.",
}),
streamShardCount: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Namespace: constants.Loki,
Name: "stream_sharding_count",
Help: "Total number of times the distributor has sharded streams",
}),
kafkaAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
Name: "kafka_appends_total",
Help: "The total number of appends sent to kafka ingest path.",
}, []string{"partition", "status"}),
kafkaWriteLatency: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{
Name: "kafka_latency_seconds",
Help: "Latency to write an incoming request to the ingest storage.",
NativeHistogramBucketFactor: 1.1,
NativeHistogramMinResetDuration: 1 * time.Hour,
NativeHistogramMaxBucketNumber: 100,
Buckets: prometheus.DefBuckets,
}),
kafkaWriteBytesTotal: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "kafka_sent_bytes_total",
Help: "Total number of bytes sent to the ingest storage.",
}),
kafkaRecordsPerRequest: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{
Name: "kafka_records_per_write_request",
Help: "The number of records a single per-partition write request has been split into.",
Buckets: prometheus.ExponentialBuckets(1, 2, 8),
}),
writeFailuresManager: writefailures.NewManager(logger, registerer, cfg.WriteFailuresLogging, configs, "distributor"),
kafkaWriter: kafkaWriter,
partitionRing: partitionRing,
}
if overrides.IngestionRateStrategy() == validation.GlobalIngestionRateStrategy {
d.rateLimitStrat = validation.GlobalIngestionRateStrategy
distributorsRing, distributorsLifecycler, err = newRingAndLifecycler(cfg.DistributorRing, d.healthyInstancesCount, logger, registerer, metricsNamespace)
if err != nil {
return nil, err
}
servs = append(servs, distributorsLifecycler, distributorsRing)
ingestionRateStrategy = newGlobalIngestionRateStrategy(overrides, d)
} else {
ingestionRateStrategy = newLocalIngestionRateStrategy(overrides)
}
d.ingestionRateLimiter = limiter.NewRateLimiter(ingestionRateStrategy, 10*time.Second)
d.distributorsRing = distributorsRing
d.distributorsLifecycler = distributorsLifecycler
d.replicationFactor.Set(float64(ingestersRing.ReplicationFactor()))
rfStats.Set(int64(ingestersRing.ReplicationFactor()))
rs := NewRateStore(
d.cfg.RateStore,
ingestersRing,
clientpool.NewPool(
"rate-store",
clientCfg.PoolConfig,
ingestersRing,
ring_client.PoolAddrFunc(internalFactory),
logger,
metricsNamespace,
),
overrides,
registerer,
)
d.rateStore = rs
servs = append(servs, d.pool, rs)
d.subservices, err = services.NewManager(servs...)
if err != nil {
return nil, errors.Wrap(err, "services manager")
}
d.subservicesWatcher = services.NewFailureWatcher()
d.subservicesWatcher.WatchManager(d.subservices)
d.Service = services.NewBasicService(d.starting, d.running, d.stopping)
return d, nil
}
func (d *Distributor) starting(ctx context.Context) error {
d.ingesterTaskWg.Add(d.cfg.PushWorkerCount)
for i := 0; i < d.cfg.PushWorkerCount; i++ {
go d.pushIngesterWorker()
}
return services.StartManagerAndAwaitHealthy(ctx, d.subservices)
}
func (d *Distributor) running(ctx context.Context) error {
defer func() {
close(d.ingesterTasks)
d.ingesterTaskWg.Wait()
}()
select {
case <-ctx.Done():
return nil
case err := <-d.subservicesWatcher.Chan():
return errors.Wrap(err, "distributor subservice failed")
}
}
func (d *Distributor) stopping(_ error) error {
if d.kafkaWriter != nil {
d.kafkaWriter.Close()
}
return services.StopManagerAndAwaitStopped(context.Background(), d.subservices)
}
type KeyedStream struct {
HashKey uint32
Stream logproto.Stream
}
// TODO taken from Cortex, see if we can refactor out an usable interface.
type streamTracker struct {
KeyedStream
minSuccess int
maxFailures int
succeeded atomic.Int32
failed atomic.Int32
}
// TODO taken from Cortex, see if we can refactor out an usable interface.
type pushTracker struct {
streamsPending atomic.Int32
streamsFailed atomic.Int32
done chan struct{}
err chan error
}
// doneWithResult records the result of a stream push.
// If err is nil, the stream push is considered successful.
// If err is not nil, the stream push is considered failed.
func (p *pushTracker) doneWithResult(err error) {
if err == nil {
if p.streamsPending.Dec() == 0 {
p.done <- struct{}{}
}
} else {
if p.streamsFailed.Inc() == 1 {
p.err <- err
}
}
}
// Push a set of streams.
// The returned error is the last one seen.
func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*logproto.PushResponse, error) {
tenantID, err := tenant.TenantID(ctx)
if err != nil {
return nil, err
}
// Return early if request does not contain any streams
if len(req.Streams) == 0 {
return &logproto.PushResponse{}, nil
}
// First we flatten out the request into a list of samples.
// We use the heuristic of 1 sample per TS to size the array.
// We also work out the hash value at the same time.
streams := make([]KeyedStream, 0, len(req.Streams))
validatedLineSize := 0
validatedLineCount := 0
var validationErrors util.GroupedErrors
validationContext := d.validator.getValidationContextForTime(time.Now(), tenantID)
func() {
sp := opentracing.SpanFromContext(ctx)
if sp != nil {
sp.LogKV("event", "start to validate request")
defer func() {
sp.LogKV("event", "finished to validate request")
}()
}
for _, stream := range req.Streams {
// Return early if stream does not contain any entries
if len(stream.Entries) == 0 {
continue
}
// Truncate first so subsequent steps have consistent line lengths
d.truncateLines(validationContext, &stream)
var lbs labels.Labels
lbs, stream.Labels, stream.Hash, err = d.parseStreamLabels(validationContext, stream.Labels, stream)
if err != nil {
d.writeFailuresManager.Log(tenantID, err)
validationErrors.Add(err)
validation.DiscardedSamples.WithLabelValues(validation.InvalidLabels, tenantID).Add(float64(len(stream.Entries)))
bytes := 0
for _, e := range stream.Entries {
bytes += len(e.Line)
}
validation.DiscardedBytes.WithLabelValues(validation.InvalidLabels, tenantID).Add(float64(bytes))
continue
}
n := 0
pushSize := 0
prevTs := stream.Entries[0].Timestamp
shouldDiscoverLevels := validationContext.allowStructuredMetadata && validationContext.discoverLogLevels
levelFromLabel, hasLevelLabel := hasAnyLevelLabels(lbs)
for _, entry := range stream.Entries {
if err := d.validator.ValidateEntry(ctx, validationContext, lbs, entry); err != nil {
d.writeFailuresManager.Log(tenantID, err)
validationErrors.Add(err)
continue
}
structuredMetadata := logproto.FromLabelAdaptersToLabels(entry.StructuredMetadata)
if shouldDiscoverLevels {
var logLevel string
if hasLevelLabel {
logLevel = levelFromLabel
} else if levelFromMetadata, ok := hasAnyLevelLabels(structuredMetadata); ok {
logLevel = levelFromMetadata
} else {
logLevel = detectLogLevelFromLogEntry(entry, structuredMetadata)
}
if logLevel != constants.LogLevelUnknown && logLevel != "" {
entry.StructuredMetadata = append(entry.StructuredMetadata, logproto.LabelAdapter{
Name: constants.LevelLabel,
Value: logLevel,
})
}
}
stream.Entries[n] = entry
// If configured for this tenant, increment duplicate timestamps. Note, this is imperfect
// since Loki will accept out of order writes it doesn't account for separate
// pushes with overlapping time ranges having entries with duplicate timestamps
if validationContext.incrementDuplicateTimestamps && n != 0 {
// Traditional logic for Loki is that 2 lines with the same timestamp and
// exact same content will be de-duplicated, (i.e. only one will be stored, others dropped)
// To maintain this behavior, only increment the timestamp if the log content is different
if stream.Entries[n-1].Line != entry.Line && (entry.Timestamp == prevTs || entry.Timestamp == stream.Entries[n-1].Timestamp) {
stream.Entries[n].Timestamp = stream.Entries[n-1].Timestamp.Add(1 * time.Nanosecond)
} else {
prevTs = entry.Timestamp
}
}
n++
validatedLineSize += len(entry.Line)
validatedLineCount++
pushSize += len(entry.Line)
}
stream.Entries = stream.Entries[:n]
if len(stream.Entries) == 0 {
// Empty stream after validating all the entries
continue
}
shardStreamsCfg := d.validator.Limits.ShardStreams(tenantID)
if shardStreamsCfg.Enabled {
streams = append(streams, d.shardStream(stream, pushSize, tenantID)...)
} else {
streams = append(streams, KeyedStream{
HashKey: lokiring.TokenFor(tenantID, stream.Labels),
Stream: stream,
})
}
}
}()
var validationErr error
if validationErrors.Err() != nil {
validationErr = httpgrpc.Errorf(http.StatusBadRequest, "%s", validationErrors.Error())
}
// Return early if none of the streams contained entries
if len(streams) == 0 {
return &logproto.PushResponse{}, validationErr
}
now := time.Now()
if block, until, retStatusCode := d.validator.ShouldBlockIngestion(validationContext, now); block {
d.trackDiscardedData(ctx, req, validationContext, tenantID, validatedLineCount, validatedLineSize, validation.BlockedIngestion)
err = fmt.Errorf(validation.BlockedIngestionErrorMsg, tenantID, until.Format(time.RFC3339), retStatusCode)
d.writeFailuresManager.Log(tenantID, err)
// If the status code is 200, return success.
// Note that we still log the error and increment the metrics.
if retStatusCode == http.StatusOK {
return &logproto.PushResponse{}, nil
}
return nil, httpgrpc.Errorf(retStatusCode, "%s", err.Error())
}
if !d.ingestionRateLimiter.AllowN(now, tenantID, validatedLineSize) {
d.trackDiscardedData(ctx, req, validationContext, tenantID, validatedLineCount, validatedLineSize, validation.RateLimited)
err = fmt.Errorf(validation.RateLimitedErrorMsg, tenantID, int(d.ingestionRateLimiter.Limit(now, tenantID)), validatedLineCount, validatedLineSize)
d.writeFailuresManager.Log(tenantID, err)
// Return a 429 to indicate to the client they are being rate limited
return nil, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error())
}
// Nil check for performance reasons, to avoid dynamic lookup and/or no-op
// function calls that cannot be inlined.
if d.tee != nil {
d.tee.Duplicate(tenantID, streams)
}
const maxExpectedReplicationSet = 5 // typical replication factor 3 plus one for inactive plus one for luck
var descs [maxExpectedReplicationSet]ring.InstanceDesc
tracker := pushTracker{
done: make(chan struct{}, 1), // buffer avoids blocking if caller terminates - sendSamples() only sends once on each
err: make(chan error, 1),
}
streamsToWrite := 0
if d.cfg.IngesterEnabled {
streamsToWrite += len(streams)
}
if d.cfg.KafkaEnabled {
streamsToWrite += len(streams)
}
// We must correctly set streamsPending before beginning any writes to ensure we don't have a race between finishing all of one path before starting the other.
tracker.streamsPending.Store(int32(streamsToWrite))
if d.cfg.KafkaEnabled {
// We don't need to create a new context like the ingester writes, because we don't return unless all writes have succeeded.
d.sendStreamsToKafka(ctx, streams, tenantID, &tracker)
}
if d.cfg.IngesterEnabled {
streamTrackers := make([]streamTracker, len(streams))
streamsByIngester := map[string][]*streamTracker{}
ingesterDescs := map[string]ring.InstanceDesc{}
if err := func() error {
sp := opentracing.SpanFromContext(ctx)
if sp != nil {
sp.LogKV("event", "started to query ingesters ring")
defer func() {
sp.LogKV("event", "finished to query ingesters ring")
}()
}
for i, stream := range streams {
replicationSet, err := d.ingestersRing.Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil)
if err != nil {
return err
}
streamTrackers[i] = streamTracker{
KeyedStream: stream,
minSuccess: len(replicationSet.Instances) - replicationSet.MaxErrors,
maxFailures: replicationSet.MaxErrors,
}
for _, ingester := range replicationSet.Instances {
streamsByIngester[ingester.Addr] = append(streamsByIngester[ingester.Addr], &streamTrackers[i])
ingesterDescs[ingester.Addr] = ingester
}
}
return nil
}(); err != nil {
return nil, err
}
for ingester, streams := range streamsByIngester {
func(ingester ring.InstanceDesc, samples []*streamTracker) {
// Use a background context to make sure all ingesters get samples even if we return early
localCtx, cancel := context.WithTimeout(context.Background(), d.clientCfg.RemoteTimeout)
localCtx = user.InjectOrgID(localCtx, tenantID)
if sp := opentracing.SpanFromContext(ctx); sp != nil {
localCtx = opentracing.ContextWithSpan(localCtx, sp)
}
d.ingesterTasks <- pushIngesterTask{
ingester: ingester,
streamTracker: samples,
pushTracker: &tracker,
ctx: localCtx,
cancel: cancel,
}
}(ingesterDescs[ingester], streams)
}
}
select {
case err := <-tracker.err:
return nil, err
case <-tracker.done:
return &logproto.PushResponse{}, validationErr
case <-ctx.Done():
return nil, ctx.Err()
}
}
func (d *Distributor) trackDiscardedData(
ctx context.Context,
req *logproto.PushRequest,
validationContext validationContext,
tenantID string,
validatedLineCount int,
validatedLineSize int,
reason string,
) {
validation.DiscardedSamples.WithLabelValues(reason, tenantID).Add(float64(validatedLineCount))
validation.DiscardedBytes.WithLabelValues(reason, tenantID).Add(float64(validatedLineSize))
if d.usageTracker != nil {
for _, stream := range req.Streams {
lbs, _, _, err := d.parseStreamLabels(validationContext, stream.Labels, stream)
if err != nil {
continue
}
discardedStreamBytes := 0
for _, e := range stream.Entries {
discardedStreamBytes += len(e.Line)
}
if d.usageTracker != nil {
d.usageTracker.DiscardedBytesAdd(ctx, tenantID, reason, lbs, float64(discardedStreamBytes))
}
}
}
}
func hasAnyLevelLabels(l labels.Labels) (string, bool) {
for lbl := range allowedLabelsForLevel {
if l.Has(lbl) {
return l.Get(lbl), true
}
}
return "", false
}
// shardStream shards (divides) the given stream into N smaller streams, where
// N is the sharding size for the given stream. shardSteam returns the smaller
// streams and their associated keys for hashing to ingesters.
//
// The number of shards is limited by the number of entries.
func (d *Distributor) shardStream(stream logproto.Stream, pushSize int, tenantID string) []KeyedStream {
shardStreamsCfg := d.validator.Limits.ShardStreams(tenantID)
logger := log.With(util_log.WithUserID(tenantID, d.logger), "stream", stream.Labels)
shardCount := d.shardCountFor(logger, &stream, pushSize, tenantID, shardStreamsCfg)
if shardCount <= 1 {
return []KeyedStream{{HashKey: lokiring.TokenFor(tenantID, stream.Labels), Stream: stream}}
}
d.streamShardCount.Inc()
if shardStreamsCfg.LoggingEnabled {
level.Info(logger).Log("msg", "sharding request", "shard_count", shardCount)
}
return d.divideEntriesBetweenShards(tenantID, shardCount, shardStreamsCfg, stream)
}
func (d *Distributor) divideEntriesBetweenShards(tenantID string, totalShards int, shardStreamsCfg shardstreams.Config, stream logproto.Stream) []KeyedStream {
derivedStreams := d.createShards(stream, totalShards, tenantID, shardStreamsCfg)
for i := 0; i < len(stream.Entries); i++ {
streamIndex := i % len(derivedStreams)
entries := append(derivedStreams[streamIndex].Stream.Entries, stream.Entries[i])
derivedStreams[streamIndex].Stream.Entries = entries
}
return derivedStreams
}
func (d *Distributor) createShards(stream logproto.Stream, totalShards int, tenantID string, shardStreamsCfg shardstreams.Config) []KeyedStream {
var (
streamLabels = labelTemplate(stream.Labels, d.logger)
streamPattern = streamLabels.String()
derivedStreams = make([]KeyedStream, 0, totalShards)
streamCount = streamCount(totalShards, stream)
)
if totalShards <= 0 {
level.Error(d.logger).Log("msg", "attempt to create shard with zeroed total shards", "org_id", tenantID, "stream", stream.Labels, "entries_len", len(stream.Entries))
return derivedStreams
}
entriesPerShard := int(math.Ceil(float64(len(stream.Entries)) / float64(totalShards)))
startShard := d.shardTracker.LastShardNum(tenantID, stream.Hash)
for i := 0; i < streamCount; i++ {
shardNum := (startShard + i) % totalShards
shard := d.createShard(streamLabels, streamPattern, shardNum, entriesPerShard)
derivedStreams = append(derivedStreams, KeyedStream{
HashKey: lokiring.TokenFor(tenantID, shard.Labels),
Stream: shard,
})
if shardStreamsCfg.LoggingEnabled {
level.Info(d.logger).Log("msg", "stream derived from sharding", "src-stream", stream.Labels, "derived-stream", shard.Labels)
}
}
d.shardTracker.SetLastShardNum(tenantID, stream.Hash, startShard+streamCount)
return derivedStreams
}
func streamCount(totalShards int, stream logproto.Stream) int {
if len(stream.Entries) < totalShards {
return len(stream.Entries)
}
return totalShards
}
// labelTemplate returns a label set that includes the dummy label to be replaced
// To avoid allocations, this slice is reused when we know the stream value
func labelTemplate(lbls string, logger log.Logger) labels.Labels {
baseLbls, err := syntax.ParseLabels(lbls)
if err != nil {
level.Error(logger).Log("msg", "couldn't extract labels from stream", "stream", lbls)
return nil
}
streamLabels := make([]labels.Label, len(baseLbls)+1)
copy(streamLabels, baseLbls)
streamLabels[len(baseLbls)] = labels.Label{Name: ingester.ShardLbName, Value: ingester.ShardLbPlaceholder}
sort.Sort(labels.Labels(streamLabels))
return streamLabels
}
func (d *Distributor) createShard(lbls labels.Labels, streamPattern string, shardNumber, numOfEntries int) logproto.Stream {
shardLabel := strconv.Itoa(shardNumber)
for i := 0; i < len(lbls); i++ {
if lbls[i].Name == ingester.ShardLbName {
lbls[i].Value = shardLabel
break
}
}
return logproto.Stream{
Labels: strings.Replace(streamPattern, ingester.ShardLbPlaceholder, shardLabel, 1),
Hash: lbls.Hash(),
Entries: make([]logproto.Entry, 0, numOfEntries),
}
}
// maxT returns the highest between two given timestamps.
func maxT(t1, t2 time.Time) time.Time {
if t1.Before(t2) {
return t2
}
return t1
}
func (d *Distributor) truncateLines(vContext validationContext, stream *logproto.Stream) {
if !vContext.maxLineSizeTruncate {
return
}
var truncatedSamples, truncatedBytes int
for i, e := range stream.Entries {
if maxSize := vContext.maxLineSize; maxSize != 0 && len(e.Line) > maxSize {
stream.Entries[i].Line = e.Line[:maxSize]
truncatedSamples++
truncatedBytes += len(e.Line) - maxSize
}
}
validation.MutatedSamples.WithLabelValues(validation.LineTooLong, vContext.userID).Add(float64(truncatedSamples))
validation.MutatedBytes.WithLabelValues(validation.LineTooLong, vContext.userID).Add(float64(truncatedBytes))
}
type pushIngesterTask struct {
streamTracker []*streamTracker
pushTracker *pushTracker
ingester ring.InstanceDesc
ctx context.Context
cancel context.CancelFunc
}
func (d *Distributor) pushIngesterWorker() {
defer d.ingesterTaskWg.Done()
for task := range d.ingesterTasks {
d.sendStreams(task)
}
}
// TODO taken from Cortex, see if we can refactor out an usable interface.
func (d *Distributor) sendStreams(task pushIngesterTask) {
defer task.cancel()
err := d.sendStreamsErr(task.ctx, task.ingester, task.streamTracker)
// If we succeed, decrement each stream's pending count by one.
// If we reach the required number of successful puts on this stream, then
// decrement the number of pending streams by one.
// If we successfully push all streams to min success ingesters, wake up the
// waiting rpc so it can return early. Similarly, track the number of errors,
// and if it exceeds maxFailures shortcut the waiting rpc.
//
// The use of atomic increments here guarantees only a single sendStreams
// goroutine will write to either channel.
for i := range task.streamTracker {
if err != nil {
if task.streamTracker[i].failed.Inc() <= int32(task.streamTracker[i].maxFailures) {
continue
}
task.pushTracker.doneWithResult(err)
} else {
if task.streamTracker[i].succeeded.Inc() != int32(task.streamTracker[i].minSuccess) {
continue
}
task.pushTracker.doneWithResult(nil)
}
}
}
// TODO taken from Cortex, see if we can refactor out an usable interface.
func (d *Distributor) sendStreamsErr(ctx context.Context, ingester ring.InstanceDesc, streams []*streamTracker) error {
c, err := d.pool.GetClientFor(ingester.Addr)
if err != nil {
return err
}
req := &logproto.PushRequest{
Streams: make([]logproto.Stream, len(streams)),
}
for i, s := range streams {
req.Streams[i] = s.Stream
}
_, err = c.(logproto.PusherClient).Push(ctx, req)
d.ingesterAppends.WithLabelValues(ingester.Addr).Inc()
if err != nil {
if e, ok := status.FromError(err); ok {
switch e.Code() {
case codes.DeadlineExceeded:
d.ingesterAppendTimeouts.WithLabelValues(ingester.Addr).Inc()
}
}
}
return err
}
func (d *Distributor) sendStreamsToKafka(ctx context.Context, streams []KeyedStream, tenant string, tracker *pushTracker) {
for _, s := range streams {
go func(s KeyedStream) {
err := d.sendStreamToKafka(ctx, s, tenant)
if err != nil {
err = fmt.Errorf("failed to write stream to kafka: %w", err)
}
tracker.doneWithResult(err)
}(s)
}
}
func (d *Distributor) sendStreamToKafka(ctx context.Context, stream KeyedStream, tenant string) error {
if len(stream.Stream.Entries) == 0 {
return nil
}
/* partitionID, err := d.partitionRing.PartitionRing().ActivePartitionForKey(stream.HashKey)
if err != nil {
d.kafkaAppends.WithLabelValues("kafka", "fail").Inc()
return fmt.Errorf("failed to find active partition for stream: %w", err)
}*/
partitionID := int32(0)
startTime := time.Now()
records, err := kafka.Encode(partitionID, tenant, stream.Stream, d.cfg.KafkaConfig.ProducerMaxRecordSizeBytes)
if err != nil {
d.kafkaAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "fail").Inc()
return fmt.Errorf("failed to marshal write request to records: %w", err)
}
d.kafkaRecordsPerRequest.Observe(float64(len(records)))
produceResults := d.kafkaWriter.ProduceSync(ctx, records)
if count, sizeBytes := successfulProduceRecordsStats(produceResults); count > 0 {
d.kafkaWriteLatency.Observe(time.Since(startTime).Seconds())
d.kafkaWriteBytesTotal.Add(float64(sizeBytes))
}
var finalErr error
for _, result := range produceResults {
if result.Err != nil {
d.kafkaAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "fail").Inc()
finalErr = result.Err
} else {
d.kafkaAppends.WithLabelValues(fmt.Sprintf("partition_%d", partitionID), "success").Inc()
}
}
return finalErr
}
func successfulProduceRecordsStats(results kgo.ProduceResults) (count, sizeBytes int) {
for _, res := range results {
if res.Err == nil && res.Record != nil {
count++
sizeBytes += len(res.Record.Value)
}
}
return
}
type labelData struct {
ls labels.Labels
hash uint64
}
func (d *Distributor) parseStreamLabels(vContext validationContext, key string, stream logproto.Stream) (labels.Labels, string, uint64, error) {
if val, ok := d.labelCache.Get(key); ok {
labelVal := val.(labelData)
return labelVal.ls, labelVal.ls.String(), labelVal.hash, nil
}
ls, err := syntax.ParseLabels(key)
if err != nil {
return nil, "", 0, fmt.Errorf(validation.InvalidLabelsErrorMsg, key, err)
}