Skip to content

Commit f80d68a

Browse files
authored
feat(distributors): Use a pool of worker to push to ingesters. (#14245)
1 parent fac3177 commit f80d68a

File tree

2 files changed

+62
-12
lines changed

2 files changed

+62
-12
lines changed

docs/sources/shared/configuration.md

+4
Original file line numberDiff line numberDiff line change
@@ -2248,6 +2248,10 @@ ring:
22482248
# CLI flag: -distributor.ring.instance-interface-names
22492249
[instance_interface_names: <list of strings> | default = [<private network interfaces>]]
22502250
2251+
# Number of workers to push batches to ingesters.
2252+
# CLI flag: -distributor.push-worker-count
2253+
[push_worker_count: <int> | default = 256]
2254+
22512255
rate_store:
22522256
# The max number of concurrent requests to make to ingester stream apis
22532257
# CLI flag: -distributor.rate-store.max-request-parallelism

pkg/distributor/distributor.go

+58-12
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010
"sort"
1111
"strconv"
1212
"strings"
13+
"sync"
1314
"time"
1415
"unicode"
1516
"unsafe"
@@ -79,6 +80,7 @@ var allowedLabelsForLevel = map[string]struct{}{
7980
type Config struct {
8081
// Distributors ring
8182
DistributorRing RingConfig `yaml:"ring,omitempty"`
83+
PushWorkerCount int `yaml:"push_worker_count"`
8284

8385
// For testing.
8486
factory ring_client.PoolFactory `yaml:"-"`
@@ -102,7 +104,7 @@ func (cfg *Config) RegisterFlags(fs *flag.FlagSet) {
102104
cfg.DistributorRing.RegisterFlags(fs)
103105
cfg.RateStore.RegisterFlagsWithPrefix("distributor.rate-store", fs)
104106
cfg.WriteFailuresLogging.RegisterFlagsWithPrefix("distributor.write-failures-logging", fs)
105-
107+
fs.IntVar(&cfg.PushWorkerCount, "distributor.push-worker-count", 256, "Number of workers to push batches to ingesters.")
106108
fs.BoolVar(&cfg.KafkaEnabled, "distributor.kafka-writes-enabled", false, "Enable writes to Kafka during Push requests.")
107109
fs.BoolVar(&cfg.IngesterEnabled, "distributor.ingester-writes-enabled", true, "Enable writes to Ingesters during Push requests. Defaults to true.")
108110
}
@@ -166,7 +168,9 @@ type Distributor struct {
166168
replicationFactor prometheus.Gauge
167169
streamShardCount prometheus.Counter
168170

169-
usageTracker push.UsageTracker
171+
usageTracker push.UsageTracker
172+
ingesterTasks chan pushIngesterTask
173+
ingesterTaskWg sync.WaitGroup
170174

171175
// kafka
172176
kafkaWriter KafkaProducer
@@ -253,6 +257,7 @@ func New(
253257
rateLimitStrat: rateLimitStrat,
254258
tee: tee,
255259
usageTracker: usageTracker,
260+
ingesterTasks: make(chan pushIngesterTask),
256261
ingesterAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
257262
Namespace: constants.Loki,
258263
Name: "distributor_ingester_appends_total",
@@ -354,6 +359,15 @@ func (d *Distributor) starting(ctx context.Context) error {
354359
}
355360

356361
func (d *Distributor) running(ctx context.Context) error {
362+
ctx, cancel := context.WithCancel(ctx)
363+
defer func() {
364+
cancel()
365+
d.ingesterTaskWg.Wait()
366+
}()
367+
d.ingesterTaskWg.Add(d.cfg.PushWorkerCount)
368+
for i := 0; i < d.cfg.PushWorkerCount; i++ {
369+
go d.pushIngesterWorker(ctx)
370+
}
357371
select {
358372
case <-ctx.Done():
359373
return nil
@@ -630,15 +644,26 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
630644
}
631645

632646
for ingester, streams := range streamsByIngester {
633-
go func(ingester ring.InstanceDesc, samples []*streamTracker) {
647+
func(ingester ring.InstanceDesc, samples []*streamTracker) {
634648
// Use a background context to make sure all ingesters get samples even if we return early
635649
localCtx, cancel := context.WithTimeout(context.Background(), d.clientCfg.RemoteTimeout)
636-
defer cancel()
637650
localCtx = user.InjectOrgID(localCtx, tenantID)
638651
if sp := opentracing.SpanFromContext(ctx); sp != nil {
639652
localCtx = opentracing.ContextWithSpan(localCtx, sp)
640653
}
641-
d.sendStreams(localCtx, ingester, samples, &tracker)
654+
select {
655+
case <-ctx.Done():
656+
cancel()
657+
return
658+
case d.ingesterTasks <- pushIngesterTask{
659+
ingester: ingester,
660+
streamTracker: samples,
661+
pushTracker: &tracker,
662+
ctx: localCtx,
663+
cancel: cancel,
664+
}:
665+
return
666+
}
642667
}(ingesterDescs[ingester], streams)
643668
}
644669
}
@@ -830,9 +855,30 @@ func (d *Distributor) truncateLines(vContext validationContext, stream *logproto
830855
validation.MutatedBytes.WithLabelValues(validation.LineTooLong, vContext.userID).Add(float64(truncatedBytes))
831856
}
832857

858+
type pushIngesterTask struct {
859+
streamTracker []*streamTracker
860+
pushTracker *pushTracker
861+
ingester ring.InstanceDesc
862+
ctx context.Context
863+
cancel context.CancelFunc
864+
}
865+
866+
func (d *Distributor) pushIngesterWorker(ctx context.Context) {
867+
defer d.ingesterTaskWg.Done()
868+
for {
869+
select {
870+
case <-ctx.Done():
871+
return
872+
case task := <-d.ingesterTasks:
873+
d.sendStreams(task)
874+
}
875+
}
876+
}
877+
833878
// TODO taken from Cortex, see if we can refactor out an usable interface.
834-
func (d *Distributor) sendStreams(ctx context.Context, ingester ring.InstanceDesc, streamTrackers []*streamTracker, pushTracker *pushTracker) {
835-
err := d.sendStreamsErr(ctx, ingester, streamTrackers)
879+
func (d *Distributor) sendStreams(task pushIngesterTask) {
880+
defer task.cancel()
881+
err := d.sendStreamsErr(task.ctx, task.ingester, task.streamTracker)
836882

837883
// If we succeed, decrement each stream's pending count by one.
838884
// If we reach the required number of successful puts on this stream, then
@@ -843,17 +889,17 @@ func (d *Distributor) sendStreams(ctx context.Context, ingester ring.InstanceDes
843889
//
844890
// The use of atomic increments here guarantees only a single sendStreams
845891
// goroutine will write to either channel.
846-
for i := range streamTrackers {
892+
for i := range task.streamTracker {
847893
if err != nil {
848-
if streamTrackers[i].failed.Inc() <= int32(streamTrackers[i].maxFailures) {
894+
if task.streamTracker[i].failed.Inc() <= int32(task.streamTracker[i].maxFailures) {
849895
continue
850896
}
851-
pushTracker.doneWithResult(err)
897+
task.pushTracker.doneWithResult(err)
852898
} else {
853-
if streamTrackers[i].succeeded.Inc() != int32(streamTrackers[i].minSuccess) {
899+
if task.streamTracker[i].succeeded.Inc() != int32(task.streamTracker[i].minSuccess) {
854900
continue
855901
}
856-
pushTracker.doneWithResult(nil)
902+
task.pushTracker.doneWithResult(nil)
857903
}
858904
}
859905
}

0 commit comments

Comments
 (0)