feat: add per-tenant time sharding for long out-of-order ingestion (#14711)

pull/14792/head
Ned Andreev 7 months ago committed by GitHub
parent 766938525f
commit 0d6d68d9bf
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 12
      docs/sources/shared/configuration.md
  2. 137
      pkg/distributor/distributor.go
  3. 269
      pkg/distributor/distributor_test.go
  4. 7
      pkg/distributor/shardstreams/config.go
  5. 1
      pkg/loki/modules.go

@ -3712,6 +3712,18 @@ shard_streams:
# CLI flag: -shard-streams.enabled
[enabled: <boolean> | default = true]
# Automatically shard streams by adding a __time_shard__ label, with values
# calculated from the log timestamps divided by MaxChunkAge/2. This allows the
# out-of-order ingestion of very old logs. If both flags are enabled,
# time-based sharding will happen before rate-based sharding.
# CLI flag: -shard-streams.time-sharding-enabled
[time_sharding_enabled: <boolean> | default = false]
# Logs with timestamps that are newer than this value will not be
# time-sharded.
# CLI flag: -shard-streams.time-sharding-ignore-recent
[time_sharding_ignore_recent: <duration> | default = 40m]
# Whether to log sharding streams behavior or not. Not recommended for
# production environments.
# CLI flag: -shard-streams.logging-enabled

@ -6,6 +6,7 @@ import (
"fmt"
"math"
"net/http"
"slices"
"sort"
"strconv"
"strings"
@ -58,6 +59,8 @@ const (
ringKey = "distributor"
ringAutoForgetUnhealthyPeriods = 2
timeShardLabel = "__time_shard__"
)
var (
@ -120,6 +123,7 @@ type Distributor struct {
services.Service
cfg Config
ingesterCfg ingester.Config
logger log.Logger
clientCfg client.Config
tenantConfigs *runtime.TenantConfigs
@ -175,6 +179,7 @@ type Distributor struct {
// New a distributor creates.
func New(
cfg Config,
ingesterCfg ingester.Config,
clientCfg client.Config,
configs *runtime.TenantConfigs,
ingestersRing ring.ReadRing,
@ -233,6 +238,7 @@ func New(
d := &Distributor{
cfg: cfg,
ingesterCfg: ingesterCfg,
logger: logger,
clientCfg: clientCfg,
tenantConfigs: configs,
@ -434,10 +440,42 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
validatedLineCount := 0
var validationErrors util.GroupedErrors
validationContext := d.validator.getValidationContextForTime(time.Now(), tenantID)
now := time.Now()
validationContext := d.validator.getValidationContextForTime(now, tenantID)
levelDetector := newLevelDetector(validationContext)
shouldDiscoverLevels := levelDetector.shouldDiscoverLogLevels()
shardStreamsCfg := d.validator.Limits.ShardStreams(tenantID)
maybeShardByRate := func(stream logproto.Stream, pushSize int) {
if shardStreamsCfg.Enabled {
streams = append(streams, d.shardStream(stream, pushSize, tenantID)...)
return
}
streams = append(streams, KeyedStream{
HashKey: lokiring.TokenFor(tenantID, stream.Labels),
Stream: stream,
})
}
maybeShardStreams := func(stream logproto.Stream, labels labels.Labels, pushSize int) {
if !shardStreamsCfg.TimeShardingEnabled {
maybeShardByRate(stream, pushSize)
return
}
ignoreRecentFrom := now.Add(-shardStreamsCfg.TimeShardingIgnoreRecent)
streamsByTime, ok := shardStreamByTime(stream, labels, d.ingesterCfg.MaxChunkAge/2, ignoreRecentFrom)
if !ok {
maybeShardByRate(stream, pushSize)
return
}
for _, ts := range streamsByTime {
maybeShardByRate(ts.Stream, ts.linesTotalLen)
}
}
func() {
sp := opentracing.SpanFromContext(ctx)
if sp != nil {
@ -446,6 +484,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
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 {
@ -512,15 +551,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
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,
})
}
maybeShardStreams(stream, lbs, pushSize)
}
}()
@ -534,8 +565,6 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
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)
@ -690,6 +719,90 @@ func (d *Distributor) trackDiscardedData(
}
}
type streamWithTimeShard struct {
logproto.Stream
linesTotalLen int
}
// This should shard the stream into multiple sub-streams based on the log
// timestamps, but with no new alocations for the log entries. It will sort them
// in-place in the given stream object (so it may modify it!) and reference
// sub-slices of the same stream.Entries slice.
//
// If the second result is false, it means that either there were no logs in the
// stream, or all of the logs in the stream occurred after the given value of
// ignoreLogsFrom, so there was no need to shard - the original `streams` value
// can be used. However, due to the in-place logs sorting by their timestamp, it
// might still have been reordered.
func shardStreamByTime(stream logproto.Stream, lbls labels.Labels, timeShardLen time.Duration, ignoreLogsFrom time.Time) ([]streamWithTimeShard, bool) {
entries := stream.Entries
entriesLen := len(entries)
if entriesLen == 0 {
return nil, false
}
slices.SortStableFunc(entries, func(a, b logproto.Entry) int { return a.Timestamp.Compare(b.Timestamp) })
// Shortcut to do no work if all of the logs are recent
if entries[0].Timestamp.After(ignoreLogsFrom) {
return nil, false
}
result := make([]streamWithTimeShard, 0, (entries[entriesLen-1].Timestamp.Sub(entries[0].Timestamp)/timeShardLen)+1)
labelBuilder := labels.NewBuilder(lbls)
startIdx := 0
for startIdx < entriesLen && entries[startIdx].Timestamp.Before(ignoreLogsFrom) /* the index is changed below */ {
timeShardStart := entries[startIdx].Timestamp.Truncate(timeShardLen)
timeShardEnd := timeShardStart.Add(timeShardLen)
timeShardCutoff := timeShardEnd
if timeShardCutoff.After(ignoreLogsFrom) {
// If the time_sharding_ignore_recent is in the middle of this
// shard, we need to cut off the logs at that point.
timeShardCutoff = ignoreLogsFrom
}
endIdx := startIdx + 1
linesTotalLen := len(entries[startIdx].Line)
for ; endIdx < entriesLen && entries[endIdx].Timestamp.Before(timeShardCutoff); endIdx++ {
linesTotalLen += len(entries[endIdx].Line)
}
shardLbls := labelBuilder.Set(timeShardLabel, fmt.Sprintf("%d_%d", timeShardStart.Unix(), timeShardEnd.Unix())).Labels()
result = append(result, streamWithTimeShard{
Stream: logproto.Stream{
Labels: shardLbls.String(),
Hash: shardLbls.Hash(),
Entries: stream.Entries[startIdx:endIdx],
},
linesTotalLen: linesTotalLen,
})
startIdx = endIdx
}
if startIdx == entriesLen {
// We do not have any remaining entries
return result, true
}
// Append one last shard with all of the logs without a time shard
logsWithoutTimeShardLen := 0
for i := startIdx; i < entriesLen; i++ {
logsWithoutTimeShardLen += len(entries[i].Line)
}
return append(result, streamWithTimeShard{
Stream: logproto.Stream{
Labels: stream.Labels,
Hash: stream.Hash,
Entries: stream.Entries[startIdx:entriesLen],
},
linesTotalLen: logsWithoutTimeShardLen,
}), true
}
// 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.

@ -793,6 +793,271 @@ func TestStreamShardAcrossCalls(t *testing.T) {
})
}
func TestStreamShardByTime(t *testing.T) {
baseTimestamp := time.Date(2024, 10, 31, 12, 34, 56, 0, time.UTC)
t.Logf("Base timestamp: %s (unix %d)", baseTimestamp.Format(time.RFC3339Nano), baseTimestamp.Unix())
for _, tc := range []struct {
test string
labels string
entries []logproto.Entry
timeShardLen time.Duration
ignoreFrom time.Time
expResult []streamWithTimeShard
}{
{
test: "zero shard because no entries",
labels: "{app='myapp'}",
entries: nil,
timeShardLen: time.Hour,
expResult: nil,
},
{
test: "single shard with one entry",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(1 * time.Second),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
}}, linesTotalLen: 3},
},
},
{
test: "one entry that is ignored",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(-10 * time.Minute),
expResult: nil,
},
{
test: "single shard with two entries",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(2 * time.Second),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 6},
},
},
{
test: "one shard and another stream with original labels",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp, Line: "foo"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(1 * time.Second),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
}}, linesTotalLen: 3},
{Stream: logproto.Stream{Labels: `{app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 3},
},
},
{
test: "single shard with two entries reversed",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp, Line: "foo"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(2 * time.Second),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 6},
},
},
{
test: "two shards without a gap",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp.Add(time.Hour), Line: "baz"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(2 * time.Hour),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 6},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730379600_1730383200", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(time.Hour), Line: "baz"},
}}, linesTotalLen: 3},
},
},
{
test: "two shards with a gap",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp.Add(4 * time.Hour), Line: "baz"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(5 * time.Hour),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 6},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730390400_1730394000", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(4 * time.Hour), Line: "baz"},
}}, linesTotalLen: 3},
},
},
{
test: "bigger shard len",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp.Add(6 * time.Hour), Line: "baz"},
},
timeShardLen: 24 * time.Hour,
ignoreFrom: baseTimestamp.Add(7 * time.Hour),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730332800_1730419200", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp.Add(6 * time.Hour), Line: "baz"},
}}, linesTotalLen: 9},
},
},
{
test: "bigger shard len with some unsharded",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
{Timestamp: baseTimestamp.Add(6 * time.Hour), Line: "baz"},
},
timeShardLen: 24 * time.Hour,
ignoreFrom: baseTimestamp.Add(5 * time.Hour),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730332800_1730419200", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp, Line: "foo"},
{Timestamp: baseTimestamp.Add(time.Second), Line: "bar"},
}}, linesTotalLen: 6},
{Stream: logproto.Stream{Labels: `{app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(6 * time.Hour), Line: "baz"},
}}, linesTotalLen: 3},
},
},
{
test: "longer messy gaps",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "11"},
{Timestamp: baseTimestamp, Line: "13"},
{Timestamp: baseTimestamp, Line: "14"},
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "12"},
{Timestamp: baseTimestamp.Add(2 * time.Hour), Line: "32"},
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(2 * time.Hour), Line: "31"},
{Timestamp: baseTimestamp.Add(5 * time.Hour), Line: "41"},
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(time.Hour), Line: "21"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Add(7 * time.Hour),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "11"},
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "12"},
{Timestamp: baseTimestamp, Line: "13"},
{Timestamp: baseTimestamp, Line: "14"},
}}, linesTotalLen: 8},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730379600_1730383200", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(time.Hour), Line: "21"},
}}, linesTotalLen: 2},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730383200_1730386800", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(2 * time.Hour), Line: "31"},
{Timestamp: baseTimestamp.Add(2 * time.Hour), Line: "32"},
}}, linesTotalLen: 4},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730394000_1730397600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Add(5 * time.Hour), Line: "41"},
}}, linesTotalLen: 2},
},
},
{
test: "longer messy with a couple ofc unsharded",
labels: `{app="myapp"}`,
entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "11"},
{Timestamp: baseTimestamp, Line: "13"},
{Timestamp: baseTimestamp, Line: "14"},
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "12"},
{Timestamp: baseTimestamp.Add(2 * time.Hour), Line: "32"},
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(2 * time.Hour), Line: "31"},
{Timestamp: baseTimestamp.Add(5 * time.Hour), Line: "41"},
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(time.Hour), Line: "21"},
},
timeShardLen: time.Hour,
ignoreFrom: baseTimestamp.Truncate(time.Hour).Add(1*time.Hour + 35*time.Minute),
expResult: []streamWithTimeShard{
{Stream: logproto.Stream{Labels: `{__time_shard__="1730376000_1730379600", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "11"},
{Timestamp: baseTimestamp.Truncate(time.Hour), Line: "12"},
{Timestamp: baseTimestamp, Line: "13"},
{Timestamp: baseTimestamp, Line: "14"},
}}, linesTotalLen: 8},
{Stream: logproto.Stream{Labels: `{__time_shard__="1730379600_1730383200", app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(time.Hour), Line: "21"},
}}, linesTotalLen: 2},
{Stream: logproto.Stream{Labels: `{app="myapp"}`, Entries: []logproto.Entry{
{Timestamp: baseTimestamp.Truncate(time.Hour).Add(2 * time.Hour), Line: "31"},
{Timestamp: baseTimestamp.Add(2 * time.Hour), Line: "32"},
{Timestamp: baseTimestamp.Add(5 * time.Hour), Line: "41"},
}}, linesTotalLen: 6},
},
},
} {
t.Run(tc.test, func(t *testing.T) {
lbls, err := syntax.ParseLabels(tc.labels)
require.NoError(t, err)
stream := logproto.Stream{
Labels: tc.labels,
Hash: lbls.Hash(),
Entries: tc.entries,
}
shardedStreams, ok := shardStreamByTime(stream, lbls, tc.timeShardLen, tc.ignoreFrom)
if tc.expResult == nil {
assert.False(t, ok)
assert.Nil(t, shardedStreams)
return
}
require.True(t, ok)
require.Len(t, shardedStreams, len(tc.expResult))
for i, ss := range shardedStreams {
assert.Equal(t, tc.expResult[i].linesTotalLen, ss.linesTotalLen)
assert.Equal(t, tc.expResult[i].Labels, ss.Labels)
assert.EqualValues(t, tc.expResult[i].Entries, ss.Entries)
}
})
}
}
func generateEntries(n int) []logproto.Entry {
var entries []logproto.Entry
for i := 0; i < n; i++ {
@ -1383,7 +1648,9 @@ func prepare(t *testing.T, numDistributors, numIngesters int, limits *validation
overrides, err := validation.NewOverrides(*limits, nil)
require.NoError(t, err)
d, err := New(distributorConfig, clientConfig, runtime.DefaultTenantConfigs(), ingestersRing, partitionRingReader, overrides, prometheus.NewPedanticRegistry(), constants.Loki, nil, nil, log.NewNopLogger())
ingesterConfig := ingester.Config{MaxChunkAge: 2 * time.Hour}
d, err := New(distributorConfig, ingesterConfig, clientConfig, runtime.DefaultTenantConfigs(), ingestersRing, partitionRingReader, overrides, prometheus.NewPedanticRegistry(), constants.Loki, nil, nil, log.NewNopLogger())
require.NoError(t, err)
require.NoError(t, services.StartAndAwaitRunning(context.Background(), d))
distributors[i] = d

@ -2,6 +2,7 @@ package shardstreams
import (
"flag"
"time"
"github.com/grafana/loki/v3/pkg/util/flagext"
)
@ -9,6 +10,10 @@ import (
type Config struct {
Enabled bool `yaml:"enabled" json:"enabled" doc:"description=Automatically shard streams to keep them under the per-stream rate limit. Sharding is dictated by the desired rate."`
TimeShardingEnabled bool `yaml:"time_sharding_enabled" json:"time_sharding_enabled" doc:"description=Automatically shard streams by adding a __time_shard__ label, with values calculated from the log timestamps divided by MaxChunkAge/2. This allows the out-of-order ingestion of very old logs. If both flags are enabled, time-based sharding will happen before rate-based sharding."`
TimeShardingIgnoreRecent time.Duration `yaml:"time_sharding_ignore_recent" json:"time_sharding_ignore_recent" doc:"description=Logs with timestamps that are newer than this value will not be time-sharded."`
LoggingEnabled bool `yaml:"logging_enabled" json:"logging_enabled" doc:"description=Whether to log sharding streams behavior or not. Not recommended for production environments."`
// DesiredRate is the threshold used to shard the stream into smaller pieces.
@ -18,6 +23,8 @@ type Config struct {
func (cfg *Config) RegisterFlagsWithPrefix(prefix string, fs *flag.FlagSet) {
fs.BoolVar(&cfg.Enabled, prefix+".enabled", true, "Automatically shard streams to keep them under the per-stream rate limit")
fs.BoolVar(&cfg.TimeShardingEnabled, prefix+".time-sharding-enabled", false, "Automatically shard streams by time (in MaxChunkAge/2 buckets), to allow out-of-order ingestion of very old logs.")
fs.DurationVar(&cfg.TimeShardingIgnoreRecent, prefix+".time-sharding-ignore-recent", 40*time.Minute, "Logs with timestamps that are newer than this value will not be time-sharded.")
fs.BoolVar(&cfg.LoggingEnabled, prefix+".logging-enabled", false, "Enable logging when sharding streams")
cfg.DesiredRate.Set("1536KB") //nolint:errcheck
fs.Var(&cfg.DesiredRate, prefix+".desired-rate", "threshold used to cut a new shard. Default (1536KB) means if a rate is above 1536KB/s, it will be sharded.")

@ -330,6 +330,7 @@ func (t *Loki) initDistributor() (services.Service, error) {
logger := log.With(util_log.Logger, "component", "distributor")
t.distributor, err = distributor.New(
t.Cfg.Distributor,
t.Cfg.Ingester,
t.Cfg.IngesterClient,
t.tenantConfigs,
t.ring,

Loading…
Cancel
Save