Skip to content

Commit 2ac9185

Browse files
feat: Add backoff to flush op (#13140)
1 parent 1b7071c commit 2ac9185

File tree

6 files changed

+241
-59
lines changed

6 files changed

+241
-59
lines changed

docs/sources/configure/_index.md

+41-25
Original file line numberDiff line numberDiff line change
@@ -944,10 +944,6 @@ alertmanager_client:
944944
# values:
945945
#
946946
# Secure Ciphers:
947-
# - TLS_RSA_WITH_AES_128_CBC_SHA
948-
# - TLS_RSA_WITH_AES_256_CBC_SHA
949-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
950-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
951947
# - TLS_AES_128_GCM_SHA256
952948
# - TLS_AES_256_GCM_SHA384
953949
# - TLS_CHACHA20_POLY1305_SHA256
@@ -965,7 +961,11 @@ alertmanager_client:
965961
# Insecure Ciphers:
966962
# - TLS_RSA_WITH_RC4_128_SHA
967963
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
964+
# - TLS_RSA_WITH_AES_128_CBC_SHA
965+
# - TLS_RSA_WITH_AES_256_CBC_SHA
968966
# - TLS_RSA_WITH_AES_128_CBC_SHA256
967+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
968+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
969969
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
970970
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
971971
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
@@ -1219,10 +1219,6 @@ evaluation:
12191219
# values:
12201220
#
12211221
# Secure Ciphers:
1222-
# - TLS_RSA_WITH_AES_128_CBC_SHA
1223-
# - TLS_RSA_WITH_AES_256_CBC_SHA
1224-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
1225-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
12261222
# - TLS_AES_128_GCM_SHA256
12271223
# - TLS_AES_256_GCM_SHA384
12281224
# - TLS_CHACHA20_POLY1305_SHA256
@@ -1240,7 +1236,11 @@ evaluation:
12401236
# Insecure Ciphers:
12411237
# - TLS_RSA_WITH_RC4_128_SHA
12421238
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
1239+
# - TLS_RSA_WITH_AES_128_CBC_SHA
1240+
# - TLS_RSA_WITH_AES_256_CBC_SHA
12431241
# - TLS_RSA_WITH_AES_128_CBC_SHA256
1242+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
1243+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
12441244
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
12451245
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
12461246
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
@@ -1442,7 +1442,23 @@ lifecycler:
14421442
# CLI flag: -ingester.flush-check-period
14431443
[flush_check_period: <duration> | default = 30s]
14441444
1445-
# The timeout before a flush is cancelled.
1445+
flush_op_backoff:
1446+
# Minimum backoff period when a flush fails. Each concurrent flush has its own
1447+
# backoff, see `ingester.concurrent-flushes`.
1448+
# CLI flag: -ingester.flush-op-backoff-min-period
1449+
[min_period: <duration> | default = 10s]
1450+
1451+
# Maximum backoff period when a flush fails. Each concurrent flush has its own
1452+
# backoff, see `ingester.concurrent-flushes`.
1453+
# CLI flag: -ingester.flush-op-backoff-max-period
1454+
[max_period: <duration> | default = 1m]
1455+
1456+
# Maximum retries for failed flushes.
1457+
# CLI flag: -ingester.flush-op-backoff-retries
1458+
[max_retries: <int> | default = 10]
1459+
1460+
# The timeout for an individual flush. Will be retried up to
1461+
# `flush-op-backoff-retries` times.
14461462
# CLI flag: -ingester.flush-op-timeout
14471463
[flush_op_timeout: <duration> | default = 10m]
14481464

@@ -3330,10 +3346,6 @@ Configuration for an ETCD v3 client. Only applies if the selected kvstore is `et
33303346
# Override the default cipher suite list (separated by commas). Allowed values:
33313347
#
33323348
# Secure Ciphers:
3333-
# - TLS_RSA_WITH_AES_128_CBC_SHA
3334-
# - TLS_RSA_WITH_AES_256_CBC_SHA
3335-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3336-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
33373349
# - TLS_AES_128_GCM_SHA256
33383350
# - TLS_AES_256_GCM_SHA384
33393351
# - TLS_CHACHA20_POLY1305_SHA256
@@ -3351,7 +3363,11 @@ Configuration for an ETCD v3 client. Only applies if the selected kvstore is `et
33513363
# Insecure Ciphers:
33523364
# - TLS_RSA_WITH_RC4_128_SHA
33533365
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
3366+
# - TLS_RSA_WITH_AES_128_CBC_SHA
3367+
# - TLS_RSA_WITH_AES_256_CBC_SHA
33543368
# - TLS_RSA_WITH_AES_128_CBC_SHA256
3369+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3370+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
33553371
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
33563372
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
33573373
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
@@ -3538,10 +3554,6 @@ When a memberlist config with atleast 1 join_members is defined, kvstore of type
35383554
# Override the default cipher suite list (separated by commas). Allowed values:
35393555
#
35403556
# Secure Ciphers:
3541-
# - TLS_RSA_WITH_AES_128_CBC_SHA
3542-
# - TLS_RSA_WITH_AES_256_CBC_SHA
3543-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3544-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
35453557
# - TLS_AES_128_GCM_SHA256
35463558
# - TLS_AES_256_GCM_SHA384
35473559
# - TLS_CHACHA20_POLY1305_SHA256
@@ -3559,7 +3571,11 @@ When a memberlist config with atleast 1 join_members is defined, kvstore of type
35593571
# Insecure Ciphers:
35603572
# - TLS_RSA_WITH_RC4_128_SHA
35613573
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
3574+
# - TLS_RSA_WITH_AES_128_CBC_SHA
3575+
# - TLS_RSA_WITH_AES_256_CBC_SHA
35623576
# - TLS_RSA_WITH_AES_128_CBC_SHA256
3577+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3578+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
35633579
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
35643580
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
35653581
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
@@ -3672,10 +3688,6 @@ backoff_config:
36723688
# Override the default cipher suite list (separated by commas). Allowed values:
36733689
#
36743690
# Secure Ciphers:
3675-
# - TLS_RSA_WITH_AES_128_CBC_SHA
3676-
# - TLS_RSA_WITH_AES_256_CBC_SHA
3677-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3678-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
36793691
# - TLS_AES_128_GCM_SHA256
36803692
# - TLS_AES_256_GCM_SHA384
36813693
# - TLS_CHACHA20_POLY1305_SHA256
@@ -3693,7 +3705,11 @@ backoff_config:
36933705
# Insecure Ciphers:
36943706
# - TLS_RSA_WITH_RC4_128_SHA
36953707
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
3708+
# - TLS_RSA_WITH_AES_128_CBC_SHA
3709+
# - TLS_RSA_WITH_AES_256_CBC_SHA
36963710
# - TLS_RSA_WITH_AES_128_CBC_SHA256
3711+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3712+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
36973713
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
36983714
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
36993715
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
@@ -3754,10 +3770,6 @@ The TLS configuration.
37543770
# Override the default cipher suite list (separated by commas). Allowed values:
37553771
#
37563772
# Secure Ciphers:
3757-
# - TLS_RSA_WITH_AES_128_CBC_SHA
3758-
# - TLS_RSA_WITH_AES_256_CBC_SHA
3759-
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3760-
# - TLS_RSA_WITH_AES_256_GCM_SHA384
37613773
# - TLS_AES_128_GCM_SHA256
37623774
# - TLS_AES_256_GCM_SHA384
37633775
# - TLS_CHACHA20_POLY1305_SHA256
@@ -3775,7 +3787,11 @@ The TLS configuration.
37753787
# Insecure Ciphers:
37763788
# - TLS_RSA_WITH_RC4_128_SHA
37773789
# - TLS_RSA_WITH_3DES_EDE_CBC_SHA
3790+
# - TLS_RSA_WITH_AES_128_CBC_SHA
3791+
# - TLS_RSA_WITH_AES_256_CBC_SHA
37783792
# - TLS_RSA_WITH_AES_128_CBC_SHA256
3793+
# - TLS_RSA_WITH_AES_128_GCM_SHA256
3794+
# - TLS_RSA_WITH_AES_256_GCM_SHA384
37793795
# - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA
37803796
# - TLS_ECDHE_RSA_WITH_RC4_128_SHA
37813797
# - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA

pkg/ingester/flush.go

+28-9
Original file line numberDiff line numberDiff line change
@@ -7,15 +7,16 @@ import (
77
"sync"
88
"time"
99

10+
"github.com/go-kit/log"
1011
"github.com/go-kit/log/level"
12+
"github.com/grafana/dskit/backoff"
13+
"github.com/grafana/dskit/tenant"
1114
"github.com/grafana/dskit/user"
1215
"github.com/prometheus/client_golang/prometheus"
1316
"github.com/prometheus/common/model"
1417
"github.com/prometheus/prometheus/model/labels"
1518
"golang.org/x/net/context"
1619

17-
"github.com/grafana/dskit/tenant"
18-
1920
"github.com/grafana/loki/pkg/chunkenc"
2021
"github.com/grafana/loki/pkg/storage/chunk"
2122
"github.com/grafana/loki/pkg/util"
@@ -126,8 +127,9 @@ func (i *Ingester) sweepStream(instance *instance, stream *stream, immediate boo
126127
}
127128

128129
func (i *Ingester) flushLoop(j int) {
130+
l := log.With(util_log.Logger, "loop", j)
129131
defer func() {
130-
level.Debug(util_log.Logger).Log("msg", "Ingester.flushLoop() exited")
132+
level.Debug(l).Log("msg", "Ingester.flushLoop() exited")
131133
i.flushQueuesDone.Done()
132134
}()
133135

@@ -138,9 +140,10 @@ func (i *Ingester) flushLoop(j int) {
138140
}
139141
op := o.(*flushOp)
140142

141-
err := i.flushUserSeries(op.userID, op.fp, op.immediate)
143+
m := util_log.WithUserID(op.userID, l)
144+
err := i.flushOp(m, op)
142145
if err != nil {
143-
level.Error(util_log.WithUserID(op.userID, util_log.Logger)).Log("msg", "failed to flush", "err", err)
146+
level.Error(m).Log("msg", "failed to flush", "err", err)
144147
}
145148

146149
// If we're exiting & we failed to flush, put the failed operation
@@ -152,7 +155,23 @@ func (i *Ingester) flushLoop(j int) {
152155
}
153156
}
154157

155-
func (i *Ingester) flushUserSeries(userID string, fp model.Fingerprint, immediate bool) error {
158+
func (i *Ingester) flushOp(l log.Logger, op *flushOp) error {
159+
ctx, cancelFunc := context.WithCancel(context.Background())
160+
defer cancelFunc()
161+
162+
b := backoff.New(ctx, i.cfg.FlushOpBackoff)
163+
for b.Ongoing() {
164+
err := i.flushUserSeries(ctx, op.userID, op.fp, op.immediate)
165+
if err == nil {
166+
break
167+
}
168+
level.Error(l).Log("msg", "failed to flush", "retries", b.NumRetries(), "err", err)
169+
b.Wait()
170+
}
171+
return b.Err()
172+
}
173+
174+
func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model.Fingerprint, immediate bool) error {
156175
instance, ok := i.getInstanceByID(userID)
157176
if !ok {
158177
return nil
@@ -166,9 +185,9 @@ func (i *Ingester) flushUserSeries(userID string, fp model.Fingerprint, immediat
166185
lbs := labels.String()
167186
level.Info(util_log.Logger).Log("msg", "flushing stream", "user", userID, "fp", fp, "immediate", immediate, "num_chunks", len(chunks), "labels", lbs)
168187

169-
ctx := user.InjectOrgID(context.Background(), userID)
170-
ctx, cancel := context.WithTimeout(ctx, i.cfg.FlushOpTimeout)
171-
defer cancel()
188+
ctx = user.InjectOrgID(ctx, userID)
189+
ctx, cancelFunc := context.WithTimeout(ctx, i.cfg.FlushOpTimeout)
190+
defer cancelFunc()
172191
err := i.flushChunks(ctx, fp, labels, chunks, chunkMtx)
173192
if err != nil {
174193
return fmt.Errorf("failed to flush chunks: %w, num_chunks: %d, labels: %s", err, len(chunks), lbs)

pkg/ingester/flush_test.go

+66
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,7 @@
11
package ingester
22

33
import (
4+
"errors"
45
"fmt"
56
"os"
67
"sort"
@@ -100,6 +101,67 @@ func Benchmark_FlushLoop(b *testing.B) {
100101
}
101102
}
102103

104+
func Test_FlushOp(t *testing.T) {
105+
t.Run("no error", func(t *testing.T) {
106+
cfg := defaultIngesterTestConfig(t)
107+
cfg.FlushOpBackoff.MinBackoff = time.Second
108+
cfg.FlushOpBackoff.MaxBackoff = 10 * time.Second
109+
cfg.FlushOpBackoff.MaxRetries = 1
110+
cfg.FlushCheckPeriod = 100 * time.Millisecond
111+
112+
_, ing := newTestStore(t, cfg, nil)
113+
114+
ctx := user.InjectOrgID(context.Background(), "foo")
115+
ins, err := ing.GetOrCreateInstance("foo")
116+
require.NoError(t, err)
117+
118+
lbs := makeRandomLabels()
119+
req := &logproto.PushRequest{Streams: []logproto.Stream{{
120+
Labels: lbs.String(),
121+
Entries: entries(5, time.Now()),
122+
}}}
123+
require.NoError(t, ins.Push(ctx, req))
124+
125+
time.Sleep(cfg.FlushCheckPeriod)
126+
require.NoError(t, ing.flushOp(gokitlog.NewNopLogger(), &flushOp{
127+
immediate: true,
128+
userID: "foo",
129+
fp: ins.getHashForLabels(lbs),
130+
}))
131+
})
132+
133+
t.Run("max retries exceeded", func(t *testing.T) {
134+
cfg := defaultIngesterTestConfig(t)
135+
cfg.FlushOpBackoff.MinBackoff = time.Second
136+
cfg.FlushOpBackoff.MaxBackoff = 10 * time.Second
137+
cfg.FlushOpBackoff.MaxRetries = 1
138+
cfg.FlushCheckPeriod = 100 * time.Millisecond
139+
140+
store, ing := newTestStore(t, cfg, nil)
141+
store.onPut = func(_ context.Context, _ []chunk.Chunk) error {
142+
return errors.New("failed to write chunks")
143+
}
144+
145+
ctx := user.InjectOrgID(context.Background(), "foo")
146+
ins, err := ing.GetOrCreateInstance("foo")
147+
require.NoError(t, err)
148+
149+
lbs := makeRandomLabels()
150+
req := &logproto.PushRequest{Streams: []logproto.Stream{{
151+
Labels: lbs.String(),
152+
Entries: entries(5, time.Now()),
153+
}}}
154+
require.NoError(t, ins.Push(ctx, req))
155+
156+
time.Sleep(cfg.FlushCheckPeriod)
157+
require.EqualError(t, ing.flushOp(gokitlog.NewNopLogger(), &flushOp{
158+
immediate: true,
159+
userID: "foo",
160+
fp: ins.getHashForLabels(lbs),
161+
}), "terminated after 1 retries")
162+
})
163+
}
164+
103165
func Test_Flush(t *testing.T) {
104166
var (
105167
store, ing = newTestStore(t, defaultIngesterTestConfig(t), nil)
@@ -295,6 +357,10 @@ func defaultIngesterTestConfig(t testing.TB) Config {
295357

296358
cfg := Config{}
297359
flagext.DefaultValues(&cfg)
360+
cfg.FlushOpBackoff.MinBackoff = 100 * time.Millisecond
361+
cfg.FlushOpBackoff.MaxBackoff = 10 * time.Second
362+
cfg.FlushOpBackoff.MaxRetries = 1
363+
cfg.FlushOpTimeout = 15 * time.Second
298364
cfg.FlushCheckPeriod = 99999 * time.Hour
299365
cfg.MaxChunkIdle = 99999 * time.Hour
300366
cfg.ConcurrentFlushes = 1

pkg/ingester/ingester.go

+14-3
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"time"
1414

1515
"github.com/go-kit/log/level"
16+
"github.com/grafana/dskit/backoff"
1617
"github.com/grafana/dskit/concurrency"
1718
"github.com/grafana/dskit/modules"
1819
"github.com/grafana/dskit/multierror"
@@ -80,6 +81,7 @@ type Config struct {
8081

8182
ConcurrentFlushes int `yaml:"concurrent_flushes"`
8283
FlushCheckPeriod time.Duration `yaml:"flush_check_period"`
84+
FlushOpBackoff backoff.Config `yaml:"flush_op_backoff"`
8385
FlushOpTimeout time.Duration `yaml:"flush_op_timeout"`
8486
RetainPeriod time.Duration `yaml:"chunk_retain_period"`
8587
MaxChunkIdle time.Duration `yaml:"chunk_idle_period"`
@@ -123,7 +125,10 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
123125
f.IntVar(&cfg.MaxTransferRetries, "ingester.max-transfer-retries", 0, "Number of times to try and transfer chunks before falling back to flushing. If set to 0 or negative value, transfers are disabled.")
124126
f.IntVar(&cfg.ConcurrentFlushes, "ingester.concurrent-flushes", 32, "How many flushes can happen concurrently from each stream.")
125127
f.DurationVar(&cfg.FlushCheckPeriod, "ingester.flush-check-period", 30*time.Second, "How often should the ingester see if there are any blocks to flush. The first flush check is delayed by a random time up to 0.8x the flush check period. Additionally, there is +/- 1% jitter added to the interval.")
126-
f.DurationVar(&cfg.FlushOpTimeout, "ingester.flush-op-timeout", 10*time.Minute, "The timeout before a flush is cancelled.")
128+
f.DurationVar(&cfg.FlushOpBackoff.MinBackoff, "ingester.flush-op-backoff-min-period", 10*time.Second, "Minimum backoff period when a flush fails. Each concurrent flush has its own backoff, see `ingester.concurrent-flushes`.")
129+
f.DurationVar(&cfg.FlushOpBackoff.MaxBackoff, "ingester.flush-op-backoff-max-period", time.Minute, "Maximum backoff period when a flush fails. Each concurrent flush has its own backoff, see `ingester.concurrent-flushes`.")
130+
f.IntVar(&cfg.FlushOpBackoff.MaxRetries, "ingester.flush-op-backoff-retries", 10, "Maximum retries for failed flushes.")
131+
f.DurationVar(&cfg.FlushOpTimeout, "ingester.flush-op-timeout", 10*time.Minute, "The timeout for an individual flush. Will be retried up to `flush-op-backoff-retries` times.")
127132
f.DurationVar(&cfg.RetainPeriod, "ingester.chunks-retain-period", 0, "How long chunks should be retained in-memory after they've been flushed.")
128133
f.DurationVar(&cfg.MaxChunkIdle, "ingester.chunks-idle-period", 30*time.Minute, "How long chunks should sit in-memory with no updates before being flushed if they don't hit the max block size. This means that half-empty chunks will still be flushed after a certain period as long as they receive no further activity.")
129134
f.IntVar(&cfg.BlockSize, "ingester.chunks-block-size", 256*1024, "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.")
@@ -151,8 +156,14 @@ func (cfg *Config) Validate() error {
151156
return err
152157
}
153158

154-
if cfg.MaxTransferRetries > 0 && cfg.WAL.Enabled {
155-
return errors.New("the use of the write ahead log (WAL) is incompatible with chunk transfers. It's suggested to use the WAL. Please try setting ingester.max-transfer-retries to 0 to disable transfers")
159+
if cfg.FlushOpBackoff.MinBackoff > cfg.FlushOpBackoff.MaxBackoff {
160+
return errors.New("invalid flush op min backoff: cannot be larger than max backoff")
161+
}
162+
if cfg.FlushOpBackoff.MaxRetries <= 0 {
163+
return fmt.Errorf("invalid flush op max retries: %d", cfg.FlushOpBackoff.MaxRetries)
164+
}
165+
if cfg.FlushOpTimeout <= 0 {
166+
return fmt.Errorf("invalid flush op timeout: %s", cfg.FlushOpTimeout)
156167
}
157168

158169
if cfg.IndexShards <= 0 {

0 commit comments

Comments
 (0)