From 219de7fd0761f1620b353d811dff518a97f6621b Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Thu, 21 Dec 2023 16:11:43 +0700 Subject: [PATCH 01/20] exponentDuration beetween retry in clickhouse output --- plugin/output/clickhouse/clickhouse.go | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index 3e82a10b5..f5d3e2488 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -5,6 +5,7 @@ import ( "context" "encoding/json" "errors" + "math" "net" "strings" "time" @@ -242,6 +243,11 @@ type Config struct { Retention cfg.Duration `json:"retention" default:"50ms" parse:"duration"` // * Retention_ time.Duration + // > @3@4@5@6 + // > + // > Exponentially increase retention beetween retries + IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > @3@4@5@6 // > // > Timeout for each insert request. @@ -492,7 +498,13 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { break } p.insertErrorsMetric.Inc() - time.Sleep(p.config.Retention_) + + retrySleep := p.config.Retention_ + if p.config.IncreaseRetentionExponentially { + retrySleep = exponentDuration(p.config.Retention_, try) + } + + time.Sleep(retrySleep) p.logger.Error("an attempt to insert a batch failed", zap.Error(err)) } if err != nil { @@ -502,6 +514,11 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } } +func exponentDuration(minRetention time.Duration, attemptNum int) time.Duration { + sleep := time.Duration(math.Pow(2, float64(attemptNum))) * minRetention + return sleep +} + func (p *Plugin) do(clickhouse Clickhouse, queryInput proto.Input) error { defer p.queriesCountMetric.Inc() From 955d9f95f877987f5ad35333e7c00f92ba2cb841 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Thu, 21 Dec 2023 16:12:21 +0700 Subject: [PATCH 02/20] GetExponentDuration in cfg package --- cfg/exponent_duration.go | 11 +++++++++ cfg/exponent_duration_test.go | 33 ++++++++++++++++++++++++++ plugin/output/clickhouse/README.md | 6 +++++ plugin/output/clickhouse/clickhouse.go | 8 +------ 4 files changed, 51 insertions(+), 7 deletions(-) create mode 100644 cfg/exponent_duration.go create mode 100644 cfg/exponent_duration_test.go diff --git a/cfg/exponent_duration.go b/cfg/exponent_duration.go new file mode 100644 index 000000000..4ed645e4a --- /dev/null +++ b/cfg/exponent_duration.go @@ -0,0 +1,11 @@ +package cfg + +import ( + "math" + "time" +) + +func GetExponentDuration(minRetention time.Duration, attemptNum int) time.Duration { + sleep := time.Duration(math.Pow(2, float64(attemptNum))) * minRetention + return sleep +} diff --git a/cfg/exponent_duration_test.go b/cfg/exponent_duration_test.go new file mode 100644 index 000000000..215c7bf64 --- /dev/null +++ b/cfg/exponent_duration_test.go @@ -0,0 +1,33 @@ +package cfg + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestGetExponentDuration(t *testing.T) { + tests := []struct { + attemptNum int + expected int64 + }{ + { + attemptNum: 0, + expected: 50, + }, + { + attemptNum: 1, + expected: 100, + }, + { + attemptNum: 10, + expected: 51200, + }, + } + + for _, test := range tests { + result := GetExponentDuration(50*time.Millisecond, test.attemptNum) + assert.Equal(t, test.expected, result.Milliseconds()) + } +} diff --git a/plugin/output/clickhouse/README.md b/plugin/output/clickhouse/README.md index cc07b5180..6aee6eaa4 100644 --- a/plugin/output/clickhouse/README.md +++ b/plugin/output/clickhouse/README.md @@ -128,6 +128,12 @@ Retention milliseconds for retry to DB.
+**`increase_retention_exponentially`** *`bool`* *`default=false`* + +Exponentially increase retention beetween retries + +
+ **`insert_timeout`** *`cfg.Duration`* *`default=10s`* Timeout for each insert request. diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index f5d3e2488..9aa638d05 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -5,7 +5,6 @@ import ( "context" "encoding/json" "errors" - "math" "net" "strings" "time" @@ -501,7 +500,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { retrySleep := p.config.Retention_ if p.config.IncreaseRetentionExponentially { - retrySleep = exponentDuration(p.config.Retention_, try) + retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) } time.Sleep(retrySleep) @@ -514,11 +513,6 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } } -func exponentDuration(minRetention time.Duration, attemptNum int) time.Duration { - sleep := time.Duration(math.Pow(2, float64(attemptNum))) * minRetention - return sleep -} - func (p *Plugin) do(clickhouse Clickhouse, queryInput proto.Input) error { defer p.queriesCountMetric.Inc() From 9d90a3b65a6fd0768abc8c616d2da556681f491e Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Wed, 18 Oct 2023 19:07:54 +0700 Subject: [PATCH 03/20] retry output plugin kafka --- plugin/output/clickhouse/clickhouse.go | 2 +- plugin/output/kafka/README.md | 19 ++++++++++ plugin/output/kafka/kafka.go | 48 ++++++++++++++++++++++++-- 3 files changed, 65 insertions(+), 4 deletions(-) diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index 9aa638d05..5aac0b4a2 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -497,6 +497,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { break } p.insertErrorsMetric.Inc() + p.logger.Error("an attempt to insert a batch failed", zap.Error(err)) retrySleep := p.config.Retention_ if p.config.IncreaseRetentionExponentially { @@ -504,7 +505,6 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } time.Sleep(retrySleep) - p.logger.Error("an attempt to insert a batch failed", zap.Error(err)) } if err != nil { p.logger.Fatal("can't insert to the table", zap.Error(err), diff --git a/plugin/output/kafka/README.md b/plugin/output/kafka/README.md index 279784df5..331d072ce 100755 --- a/plugin/output/kafka/README.md +++ b/plugin/output/kafka/README.md @@ -57,6 +57,25 @@ After this timeout the batch will be sent even if batch isn't full.
+**`retry`** *`int`* *`default=10`* + +Retries of insertion. If File.d cannot insert for this number of attempts, +File.d will fall with non-zero exit code. + +
+ +**`retention`** *`cfg.Duration`* *`default=50ms`* + +Retention milliseconds for retry. + +
+ +**`increase_retention_exponentially`** *`bool`* *`default=false`* + +Exponentially increase retention beetween retries + +
+ **`is_sasl_enabled`** *`bool`* *`default=false`* If set, the plugin will use SASL authentications mechanism. diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index a1c2c0c7a..62b1855cf 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -95,6 +95,23 @@ type Config struct { BatchFlushTimeout cfg.Duration `json:"batch_flush_timeout" default:"200ms" parse:"duration"` // * BatchFlushTimeout_ time.Duration + // > @3@4@5@6 + // > + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code. + Retry int `json:"retry" default:"10"` // * + + // > @3@4@5@6 + // > + // > Retention milliseconds for retry. + Retention cfg.Duration `json:"retention" default:"50ms" parse:"duration"` // * + Retention_ time.Duration + + // > @3@4@5@6 + // > + // > Exponentially increase retention beetween retries + IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > @3@4@5@6 // > // > If set, the plugin will use SASL authentications mechanism. @@ -149,6 +166,10 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.controller = params.Controller p.registerMetrics(params.MetricCtl) + if p.config.Retention_ < 1 { + p.logger.Fatal("'retention' can't be <1") + } + p.logger.Infof("workers count=%d, batch size=%d", p.config.WorkersCount_, p.config.BatchSize_) p.producer = NewProducer(p.config, p.logger) @@ -213,14 +234,35 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = outBuf - err := p.producer.SendMessages(data.messages[:i]) - if err != nil { + var err error + for try := 0; try < p.config.Retry; try++ { + err := p.producer.SendMessages(data.messages[:i]) + if err == nil { + break + } + errs := err.(sarama.ProducerErrors) for _, e := range errs { p.logger.Errorf("can't write batch: %s", e.Err.Error()) } p.sendErrorMetric.Add(float64(len(errs))) - p.controller.Error("some events from batch were not written") + p.logger.Error( + "an attempt to insert a batch failed", + zap.Int("retry", (try+1)), + zap.Error(err), + ) + + retrySleep := p.config.Retention_ + if p.config.IncreaseRetentionExponentially { + retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) + } + + time.Sleep(retrySleep) + } + + if err != nil { + p.logger.Fatal("can't write batch", zap.Error(err), + zap.Int("retries", p.config.Retry)) } } From 1d5ca633756d70c0b28adc662c3a5dd5a00b8689 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Mon, 23 Oct 2023 13:32:59 +0700 Subject: [PATCH 04/20] retry in postgres output plugin --- plugin/output/clickhouse/clickhouse.go | 6 ++- plugin/output/postgres/README.idoc.md | 3 ++ plugin/output/postgres/README.md | 39 ++++++++++++++++++ plugin/output/postgres/postgres.go | 57 ++++++++++++++++++++++++-- 4 files changed, 100 insertions(+), 5 deletions(-) diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index 5aac0b4a2..f54ffa6cd 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -497,7 +497,11 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { break } p.insertErrorsMetric.Inc() - p.logger.Error("an attempt to insert a batch failed", zap.Error(err)) + p.logger.Error( + "an attempt to insert a batch failed", + zap.Error(err), + zap.Int("try", try), + ) retrySleep := p.config.Retention_ if p.config.IncreaseRetentionExponentially { diff --git a/plugin/output/postgres/README.idoc.md b/plugin/output/postgres/README.idoc.md index 2037fba64..657419cef 100644 --- a/plugin/output/postgres/README.idoc.md +++ b/plugin/output/postgres/README.idoc.md @@ -3,3 +3,6 @@ ### Config params @config-params|description + +### Example +@example diff --git a/plugin/output/postgres/README.md b/plugin/output/postgres/README.md index bb14ce874..d86b4fe2c 100755 --- a/plugin/output/postgres/README.md +++ b/plugin/output/postgres/README.md @@ -50,6 +50,12 @@ Retention milliseconds for retry to DB.
+**`increase_retention_exponentially`** *`bool`* *`default=false`* + +Exponentially increase retention beetween retries + +
+ **`db_request_timeout`** *`cfg.Duration`* *`default=3000ms`* Timeout for DB requests in milliseconds. @@ -91,4 +97,37 @@ After this timeout batch will be sent even if batch isn't completed.
+### Example +**Example** +Postgres output example: +```yaml +pipelines: + example_pipeline: + input: + type: file + persistence_mode: async + watching_dir: ./ + filename_pattern: input_example.json + offsets_file: ./offsets.yaml + offsets_op: reset + output: + type: postgres + conn_string: "user=postgres host=localhost port=5432 dbname=postgres sslmode=disable pool_max_conns=10" + table: events + columns: + - name: id + type: int + - name: name + type: string + retry: 10 + retention: 1s + increase_retention_exponentially: true +``` + +input_example.json +```json +{"id":1,"name":"name1"} +{"id":2,"name":"name2"} +``` +
*Generated using [__insane-doc__](https://github.com/vitkovskii/insane-doc)* \ No newline at end of file diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index a793e577e..205190f52 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -23,6 +23,40 @@ import ( It sends the event batches to postgres db using pgx. }*/ +/*{ example +**Example** +Postgres output example: +```yaml +pipelines: + example_pipeline: + input: + type: file + persistence_mode: async + watching_dir: ./ + filename_pattern: input_example.json + offsets_file: ./offsets.yaml + offsets_op: reset + output: + type: postgres + conn_string: "user=postgres host=localhost port=5432 dbname=postgres sslmode=disable pool_max_conns=10" + table: events + columns: + - name: id + type: int + - name: name + type: string + retry: 10 + retention: 1s + increase_retention_exponentially: true +``` + +input_example.json +```json +{"id":1,"name":"name1"} +{"id":2,"name":"name2"} +``` +}*/ + var ( ErrEventDoesntHaveField = errors.New("event doesn't have field") ErrEventFieldHasWrongType = errors.New("event field has wrong type") @@ -71,9 +105,11 @@ type Plugin struct { pool PgxIface // plugin metrics + discardedEventMetric prometheus.Counter duplicatedEventMetric prometheus.Counter writtenEventMetric prometheus.Counter + insertErrorsMetric prometheus.Counter } type ConfigColumn struct { @@ -128,6 +164,11 @@ type Config struct { Retention cfg.Duration `json:"retention" default:"50ms" parse:"duration"` // * Retention_ time.Duration + // > @3@4@5@6 + // > + // > Exponentially increase retention beetween retries + IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > @3@4@5@6 // > // > Timeout for DB requests in milliseconds. @@ -182,6 +223,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.discardedEventMetric = ctl.RegisterCounter("output_postgres_event_discarded", "Total pgsql discarded messages") p.duplicatedEventMetric = ctl.RegisterCounter("output_postgres_event_duplicated", "Total pgsql duplicated messages") p.writtenEventMetric = ctl.RegisterCounter("output_postgres_event_written", "Total events written to pgsql") + p.insertErrorsMetric = ctl.RegisterCounter("output_postgres_insert_errors", "Total pgsql insert errors") } func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginParams) { @@ -192,7 +234,7 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.registerMetrics(params.MetricCtl) if len(p.config.Columns) == 0 { - p.logger.Fatal("can't start plugin, no fields in config") + p.logger.Fatal("can't start plugin, no columns in config") } if p.config.Retry < 1 { @@ -315,11 +357,18 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { } // Insert into pg with retry. - for i := p.config.Retry; i > 0; i-- { + for try := 0; try < p.config.Retry; try++ { err = p.try(query, argsSliceInterface) if err != nil { - p.logger.Errorf("can't exec query: %s", err.Error()) - time.Sleep(p.config.Retention_) + p.insertErrorsMetric.Inc() + p.logger.Errorf("can't exec query: %s (try %d)", err.Error(), try) + + retrySleep := p.config.Retention_ + if p.config.IncreaseRetentionExponentially { + retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) + } + + time.Sleep(retrySleep) continue } p.writtenEventMetric.Add(float64(len(uniqueEventsMap))) From fdc6603d47c3edaddcfc73b0837e484cb905de9e Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Mon, 23 Oct 2023 19:49:41 +0700 Subject: [PATCH 05/20] fix example config for s3 output --- plugin/output/README.md | 8 ++++---- plugin/output/s3/README.md | 8 ++++---- plugin/output/s3/TESTME.md | 8 ++++---- plugin/output/s3/s3.go | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/plugin/output/README.md b/plugin/output/README.md index daff2a16b..124405eac 100755 --- a/plugin/output/README.md +++ b/plugin/output/README.md @@ -98,12 +98,12 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 - file_plugin: + file_config: retention_interval: 10s # endpoint, access_key, secret_key, bucket are required. endpoint: "s3.fake_host.org:80" diff --git a/plugin/output/s3/README.md b/plugin/output/s3/README.md index 53bef4194..f1ae1e386 100755 --- a/plugin/output/s3/README.md +++ b/plugin/output/s3/README.md @@ -48,12 +48,12 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 - file_plugin: + file_config: retention_interval: 10s # endpoint, access_key, secret_key, bucket are required. endpoint: "s3.fake_host.org:80" diff --git a/plugin/output/s3/TESTME.md b/plugin/output/s3/TESTME.md index c57992e3b..92389406c 100644 --- a/plugin/output/s3/TESTME.md +++ b/plugin/output/s3/TESTME.md @@ -52,12 +52,12 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 - file_plugin: + file_config: retention_interval: 10s endpoint: "0.0.0.0:19001" access_key: "minio_access_key" diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index b8d0050b5..e8be5725e 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -73,12 +73,12 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 - file_plugin: + file_config: retention_interval: 10s # endpoint, access_key, secret_key, bucket are required. endpoint: "s3.fake_host.org:80" From 0293f73d624bdcb05ee7b0b20a46869b6e5b8f07 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Thu, 21 Dec 2023 16:47:59 +0700 Subject: [PATCH 06/20] use backoff --- cfg/backoff.go | 15 ++++++ cfg/exponent_duration.go | 11 ----- cfg/exponent_duration_test.go | 33 ------------- plugin/output/clickhouse/clickhouse.go | 48 +++++++++--------- plugin/output/elasticsearch/elasticsearch.go | 37 +++++++++++--- plugin/output/kafka/kafka.go | 33 +++++++------ plugin/output/postgres/postgres.go | 36 +++++++------- plugin/output/s3/s3.go | 51 ++++++++++++++------ plugin/output/splunk/splunk.go | 34 ++++++++++--- 9 files changed, 172 insertions(+), 126 deletions(-) create mode 100644 cfg/backoff.go delete mode 100644 cfg/exponent_duration.go delete mode 100644 cfg/exponent_duration_test.go diff --git a/cfg/backoff.go b/cfg/backoff.go new file mode 100644 index 000000000..c96be65e4 --- /dev/null +++ b/cfg/backoff.go @@ -0,0 +1,15 @@ +package cfg + +import ( + "time" + + "github.com/cenkalti/backoff/v3" +) + +func GetBackoff(minRetention time.Duration, multiplier float64, attemptNum uint64) backoff.BackOff { + expBackoff := backoff.NewExponentialBackOff() + expBackoff.InitialInterval = minRetention + expBackoff.Multiplier = multiplier + expBackoff.RandomizationFactor = 0.5 + return backoff.WithMaxRetries(expBackoff, attemptNum) +} diff --git a/cfg/exponent_duration.go b/cfg/exponent_duration.go deleted file mode 100644 index 4ed645e4a..000000000 --- a/cfg/exponent_duration.go +++ /dev/null @@ -1,11 +0,0 @@ -package cfg - -import ( - "math" - "time" -) - -func GetExponentDuration(minRetention time.Duration, attemptNum int) time.Duration { - sleep := time.Duration(math.Pow(2, float64(attemptNum))) * minRetention - return sleep -} diff --git a/cfg/exponent_duration_test.go b/cfg/exponent_duration_test.go deleted file mode 100644 index 215c7bf64..000000000 --- a/cfg/exponent_duration_test.go +++ /dev/null @@ -1,33 +0,0 @@ -package cfg - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" -) - -func TestGetExponentDuration(t *testing.T) { - tests := []struct { - attemptNum int - expected int64 - }{ - { - attemptNum: 0, - expected: 50, - }, - { - attemptNum: 1, - expected: 100, - }, - { - attemptNum: 10, - expected: 51200, - }, - } - - for _, test := range tests { - result := GetExponentDuration(50*time.Millisecond, test.attemptNum) - assert.Equal(t, test.expected, result.Milliseconds()) - } -} diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index f54ffa6cd..25feeac6a 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -12,6 +12,7 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/chpool" "github.com/ClickHouse/ch-go/proto" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -46,6 +47,7 @@ type Plugin struct { batcher *pipeline.Batcher ctx context.Context cancelFunc context.CancelFunc + backoff backoff.BackOff query string @@ -228,7 +230,7 @@ type Config struct { // > // > Retries of insertion. If File.d cannot insert for this number of attempts, // > File.d will fall with non-zero exit code. - Retry int `json:"retry" default:"10"` // * + Retry uint64 `json:"retry" default:"10"` // * // > @3@4@5@6 // > @@ -244,8 +246,8 @@ type Config struct { // > @3@4@5@6 // > - // > Exponentially increase retention beetween retries - IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * // > @3@4@5@6 // > @@ -410,6 +412,12 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP } } + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) + p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ PipelineName: params.PipelineName, OutputType: outPluginType, @@ -488,31 +496,27 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } }) - var err error - for try := 0; try < p.config.Retry; try++ { + p.backoff.Reset() + try := 0 + err := backoff.Retry(func() error { requestID := p.requestID.Inc() clickhouse := p.getInstance(requestID, try) - err = p.do(clickhouse, data.input) - if err == nil { - break - } - p.insertErrorsMetric.Inc() - p.logger.Error( - "an attempt to insert a batch failed", - zap.Error(err), - zap.Int("try", try), - ) - - retrySleep := p.config.Retention_ - if p.config.IncreaseRetentionExponentially { - retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) + err := p.do(clickhouse, data.input) + if err != nil { + try++ + p.insertErrorsMetric.Inc() + p.logger.Error( + "an attempt to insert a batch failed", + zap.Int("try", try), + zap.Error(err), + ) } + return err + }, p.backoff) - time.Sleep(retrySleep) - } if err != nil { p.logger.Fatal("can't insert to the table", zap.Error(err), - zap.Int("retries", p.config.Retry), + zap.Uint64("retries", p.config.Retry), zap.String("table", p.config.Table)) } } diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 3dd9e9cc3..0ed7e7092 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -9,6 +9,7 @@ import ( "sync" "time" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/logger" @@ -29,7 +30,6 @@ If a network error occurs, the batch will infinitely try to be delivered to the const ( outPluginType = "elasticsearch" NDJSONContentType = "application/x-ndjson" - retryDelay = time.Second ) var ( @@ -49,6 +49,7 @@ type Plugin struct { batcher *pipeline.Batcher controller pipeline.OutputPluginController mu *sync.Mutex + backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -139,6 +140,23 @@ type Config struct { // > Operation type to be used in batch requests. It can be `index` or `create`. Default is `index`. // > > Check out [_bulk API doc](https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html) for details. BatchOpType string `json:"batch_op_type" default:"index" options:"index|create"` // * + + // > @3@4@5@6 + // > + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code. + Retry uint64 `json:"retry" default:"0"` // * + + // > @3@4@5@6 + // > + // > Retention milliseconds for retry to DB. + Retention cfg.Duration `json:"retention" default:"1s" parse:"duration"` // * + Retention_ time.Duration + + // > @3@4@5@6 + // > + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * } type data struct { @@ -220,6 +238,12 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP ctx, cancel := context.WithCancel(context.Background()) p.cancel = cancel + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) + p.batcher.Start(ctx) } @@ -255,14 +279,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = p.appendEvent(data.outBuf, event) }) - for { + p.backoff.Reset() + backoff.Retry(func() error { + var err error if err := p.send(data.outBuf); err != nil { p.sendErrorMetric.Inc() p.logger.Error("can't send to the elastic, will try other endpoint", zap.Error(err)) - } else { - break } - } + return err + }, p.backoff) } func (p *Plugin) send(body []byte) error { @@ -279,14 +304,12 @@ func (p *Plugin) send(body []byte) error { p.setAuthHeader(req) if err := p.client.DoTimeout(req, resp, p.config.ConnectionTimeout_); err != nil { - time.Sleep(retryDelay) return fmt.Errorf("can't send batch to %s: %s", endpoint.String(), err.Error()) } respContent := resp.Body() if statusCode := resp.Header.StatusCode(); statusCode < http.StatusOK || statusCode > http.StatusAccepted { - time.Sleep(retryDelay) return fmt.Errorf("response status from %s isn't OK: status=%d, body=%s", endpoint.String(), statusCode, string(respContent)) } diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index 62b1855cf..9b98f0553 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -6,6 +6,7 @@ import ( "time" "github.com/Shopify/sarama" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -37,6 +38,7 @@ type Plugin struct { producer sarama.SyncProducer batcher *pipeline.Batcher + backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -99,7 +101,7 @@ type Config struct { // > // > Retries of insertion. If File.d cannot insert for this number of attempts, // > File.d will fall with non-zero exit code. - Retry int `json:"retry" default:"10"` // * + Retry uint64 `json:"retry" default:"10"` // * // > @3@4@5@6 // > @@ -109,8 +111,8 @@ type Config struct { // > @3@4@5@6 // > - // > Exponentially increase retention beetween retries - IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * // > @3@4@5@6 // > @@ -169,6 +171,11 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP if p.config.Retention_ < 1 { p.logger.Fatal("'retention' can't be <1") } + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) p.logger.Infof("workers count=%d, batch size=%d", p.config.WorkersCount_, p.config.BatchSize_) @@ -234,11 +241,11 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = outBuf - var err error - for try := 0; try < p.config.Retry; try++ { + p.backoff.Reset() + err := backoff.Retry(func() error { err := p.producer.SendMessages(data.messages[:i]) if err == nil { - break + return nil } errs := err.(sarama.ProducerErrors) @@ -248,21 +255,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.sendErrorMetric.Add(float64(len(errs))) p.logger.Error( "an attempt to insert a batch failed", - zap.Int("retry", (try+1)), zap.Error(err), ) - retrySleep := p.config.Retention_ - if p.config.IncreaseRetentionExponentially { - retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) - } - - time.Sleep(retrySleep) - } + return err + }, p.backoff) if err != nil { - p.logger.Fatal("can't write batch", zap.Error(err), - zap.Int("retries", p.config.Retry)) + p.logger.Error("can't write batch", zap.Error(err), + zap.Uint64("retries", p.config.Retry)) } } diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index 205190f52..e7a6e2bbc 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -8,6 +8,7 @@ import ( "time" sq "github.com/Masterminds/squirrel" + "github.com/cenkalti/backoff/v3" "github.com/jackc/pgx/v4" "github.com/jackc/pgx/v4/pgxpool" "github.com/ozontech/file.d/cfg" @@ -47,7 +48,7 @@ pipelines: type: string retry: 10 retention: 1s - increase_retention_exponentially: true + retention_exponentially_multiplier: 1.5 ``` input_example.json @@ -104,6 +105,8 @@ type Plugin struct { queryBuilder PgQueryBuilder pool PgxIface + backoff backoff.BackOff + // plugin metrics discardedEventMetric prometheus.Counter @@ -156,7 +159,7 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. - Retry int `json:"retry" default:"3"` // * + Retry uint64 `json:"retry" default:"3"` // * // > @3@4@5@6 // > @@ -166,8 +169,8 @@ type Config struct { // > @3@4@5@6 // > - // > Exponentially increase retention beetween retries - IncreaseRetentionExponentially bool `json:"increase_retention_exponentially" default:"false"` // * + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // > @3@4@5@6 // > @@ -279,6 +282,12 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP MetricCtl: params.MetricCtl, }) + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) + ctx, cancel := context.WithCancel(context.Background()) p.ctx = ctx p.cancelFunc = cancel @@ -356,24 +365,17 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { argsSliceInterface[i] = args[i-1] } - // Insert into pg with retry. - for try := 0; try < p.config.Retry; try++ { + p.backoff.Reset() + err = backoff.Retry(func() error { err = p.try(query, argsSliceInterface) if err != nil { p.insertErrorsMetric.Inc() - p.logger.Errorf("can't exec query: %s (try %d)", err.Error(), try) - - retrySleep := p.config.Retention_ - if p.config.IncreaseRetentionExponentially { - retrySleep = cfg.GetExponentDuration(p.config.Retention_, try) - } - - time.Sleep(retrySleep) - continue + p.logger.Errorf("can't exec query: %s", err.Error()) + return err } p.writtenEventMetric.Add(float64(len(uniqueEventsMap))) - break - } + return nil + }, p.backoff) if err != nil { p.pool.Close() diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index e8be5725e..d9b412a5c 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -13,6 +13,7 @@ import ( "sync" "time" + "github.com/cenkalti/backoff/v3" "github.com/minio/minio-go" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" @@ -102,16 +103,14 @@ pipelines: }*/ const ( - fileNameSeparator = "_" - attemptIntervalMin = 1 * time.Second - dirSep = "/" - StaticBucketDir = "static_buckets" - DynamicBucketDir = "dynamic_buckets" + fileNameSeparator = "_" + dirSep = "/" + StaticBucketDir = "static_buckets" + DynamicBucketDir = "dynamic_buckets" ) var ( - attemptInterval = attemptIntervalMin - compressors = map[string]func(*zap.SugaredLogger) compressor{ + compressors = map[string]func(*zap.SugaredLogger) compressor{ zipName: newZipCompressor, } ) @@ -149,6 +148,8 @@ type Plugin struct { compressor compressor + backoff backoff.BackOff + // plugin metrics sendErrorMetric prometheus.Counter uploadFileMetric *prometheus.CounterVec @@ -234,6 +235,23 @@ type Config struct { // > Sets upload timeout. UploadTimeout cfg.Duration `json:"upload_timeout" default:"1m" parse:"duration"` // * UploadTimeout_ time.Duration + + // > @3@4@5@6 + // > + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code. + Retry uint64 `json:"retry" default:"0"` // * + + // > @3@4@5@6 + // > + // > Retention milliseconds for retry to upload. + Retention cfg.Duration `json:"retention" default:"1s" parse:"duration"` // * + Retention_ time.Duration + + // > @3@4@5@6 + // > + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1.5"` // * } func (c *Config) IsMultiBucketExists(bucketName string) bool { @@ -279,6 +297,12 @@ func (p *Plugin) StartWithMinio(config pipeline.AnyConfig, params *pipeline.Outp p.config = config.(*Config) p.params = params + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) + // outPlugCount is defaultBucket + multi_buckets count, use to set maps size. outPlugCount := len(p.config.MultiBuckets) + 1 p.limiter = NewObjectStoreClientLimiter(p.config.DynamicBucketsLimit + outPlugCount) @@ -507,8 +531,8 @@ func (p *Plugin) addFileJobWithBucket(bucketName string) func(filename string) { func (p *Plugin) uploadWork() { for compressed := range p.uploadCh { - sleepTime := attemptInterval - for { + p.backoff.Reset() + backoff.Retry(func() error { p.logger.Infof("starting upload s3 object. fileName=%s, bucketName=%s", compressed.fileName, compressed.bucketName) err := p.uploadToS3(compressed) if err == nil { @@ -519,12 +543,11 @@ func (p *Plugin) uploadWork() { if err != nil && !os.IsNotExist(err) { p.logger.Panicf("could not delete file: %s, err: %s", compressed, err.Error()) } - break + return nil } - sleepTime += sleepTime - p.logger.Errorf("could not upload object: %s, next attempt in %s, error: %s", compressed, sleepTime.String(), err.Error()) - time.Sleep(sleepTime) - } + p.logger.Errorf("could not upload object: %s, error: %s", compressed, err.Error()) + return err + }, p.backoff) } } diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index 269d13425..fdc1b4aba 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -10,6 +10,7 @@ import ( "net/http" "time" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -34,6 +35,7 @@ type Plugin struct { avgEventSize int batcher *pipeline.Batcher controller pipeline.OutputPluginController + backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -82,6 +84,23 @@ type Config struct { // > After this timeout the batch will be sent even if batch isn't completed. BatchFlushTimeout cfg.Duration `json:"batch_flush_timeout" default:"200ms" parse:"duration"` // * BatchFlushTimeout_ time.Duration + + // > @3@4@5@6 + // > + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code. + Retry uint64 `json:"retry" default:"0"` // * + + // > @3@4@5@6 + // > + // > Retention milliseconds for retry to DB. + Retention cfg.Duration `json:"retention" default:"1s" parse:"duration"` // * + Retention_ time.Duration + + // > @3@4@5@6 + // > + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * } type data struct { @@ -106,6 +125,11 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config = config.(*Config) p.registerMetrics(params.MetricCtl) p.client = p.newClient(p.config.RequestTimeout_) + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ PipelineName: params.PipelineName, @@ -161,18 +185,16 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.logger.Debugf("trying to send: %s", outBuf) - for { + p.backoff.Reset() + backoff.Retry(func() error { err := p.send(outBuf) if err != nil { p.sendErrorMetric.Inc() p.logger.Errorf("can't send data to splunk address=%s: %s", p.config.Endpoint, err.Error()) - time.Sleep(time.Second) - - continue } + return err + }, p.backoff) - break - } p.logger.Debugf("successfully sent: %s", outBuf) } From 5ffda8aa58ef3951bed75efaa66a0c1752f55292 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Fri, 27 Oct 2023 16:43:45 +0700 Subject: [PATCH 07/20] fall after attempts of retry in elasticsearch, s3 and splunk --- plugin/README.md | 8 ++++---- plugin/output/clickhouse/README.md | 6 +++--- plugin/output/elasticsearch/README.md | 19 +++++++++++++++++++ plugin/output/elasticsearch/elasticsearch.go | 8 +++++++- plugin/output/kafka/README.md | 6 +++--- plugin/output/postgres/README.md | 13 +++++-------- plugin/output/s3/README.md | 19 +++++++++++++++++++ plugin/output/s3/s3.go | 10 ++++++++-- plugin/output/splunk/README.md | 19 +++++++++++++++++++ plugin/output/splunk/splunk.go | 8 +++++++- 10 files changed, 94 insertions(+), 22 deletions(-) diff --git a/plugin/README.md b/plugin/README.md index 3f8e0e06d..c9a21f163 100755 --- a/plugin/README.md +++ b/plugin/README.md @@ -759,12 +759,12 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 - file_plugin: + file_config: retention_interval: 10s # endpoint, access_key, secret_key, bucket are required. endpoint: "s3.fake_host.org:80" diff --git a/plugin/output/clickhouse/README.md b/plugin/output/clickhouse/README.md index 6aee6eaa4..241e32617 100644 --- a/plugin/output/clickhouse/README.md +++ b/plugin/output/clickhouse/README.md @@ -108,7 +108,7 @@ If the strict mode is enabled file.d fails (exit with code 1) in above examples.
-**`retry`** *`int`* *`default=10`* +**`retry`** *`uint64`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, File.d will fall with non-zero exit code. @@ -128,9 +128,9 @@ Retention milliseconds for retry to DB.
-**`increase_retention_exponentially`** *`bool`* *`default=false`* +**`retention_exponentially_multiplier`** *`float64`* *`default=1`* -Exponentially increase retention beetween retries +Multiplier for exponentially increase retention beetween retries
diff --git a/plugin/output/elasticsearch/README.md b/plugin/output/elasticsearch/README.md index a7908aa47..a263d8442 100755 --- a/plugin/output/elasticsearch/README.md +++ b/plugin/output/elasticsearch/README.md @@ -94,5 +94,24 @@ Operation type to be used in batch requests. It can be `index` or `create`. Defa
+**`retry`** *`uint64`* *`default=0`* + +Retries of insertion. If File.d cannot insert for this number of attempts, +File.d will fall with non-zero exit code. + +
+ +**`retention`** *`cfg.Duration`* *`default=1s`* + +Retention milliseconds for retry to DB. + +
+ +**`retention_exponentially_multiplier`** *`float64`* *`default=1`* + +Multiplier for exponentially increase retention beetween retries + +
+
*Generated using [__insane-doc__](https://github.com/vitkovskii/insane-doc)* \ No newline at end of file diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 0ed7e7092..5c6047d56 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -280,7 +280,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }) p.backoff.Reset() - backoff.Retry(func() error { + err := backoff.Retry(func() error { var err error if err := p.send(data.outBuf); err != nil { p.sendErrorMetric.Inc() @@ -288,6 +288,12 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } return err }, p.backoff) + + if err != nil { + p.logger.Fatal("can't send to the elastic", zap.Error(err), + zap.Uint64("retries", p.config.Retry), + ) + } } func (p *Plugin) send(body []byte) error { diff --git a/plugin/output/kafka/README.md b/plugin/output/kafka/README.md index 331d072ce..cc77cd772 100755 --- a/plugin/output/kafka/README.md +++ b/plugin/output/kafka/README.md @@ -57,7 +57,7 @@ After this timeout the batch will be sent even if batch isn't full.
-**`retry`** *`int`* *`default=10`* +**`retry`** *`uint64`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, File.d will fall with non-zero exit code. @@ -70,9 +70,9 @@ Retention milliseconds for retry.
-**`increase_retention_exponentially`** *`bool`* *`default=false`* +**`retention_exponentially_multiplier`** *`float64`* *`default=1`* -Exponentially increase retention beetween retries +Multiplier for exponentially increase retention beetween retries
diff --git a/plugin/output/postgres/README.md b/plugin/output/postgres/README.md index d86b4fe2c..3b30c4b84 100755 --- a/plugin/output/postgres/README.md +++ b/plugin/output/postgres/README.md @@ -38,7 +38,7 @@ and nullable options.
-**`retry`** *`int`* *`default=3`* +**`retry`** *`uint64`* *`default=3`* Retries of insertion. @@ -50,14 +50,11 @@ Retention milliseconds for retry to DB.
-**`increase_retention_exponentially`** *`bool`* *`default=false`* - -Exponentially increase retention beetween retries - -
- **`db_request_timeout`** *`cfg.Duration`* *`default=3000ms`* +Multiplier for exponentially increase retention beetween retries +*`cfg.Duration`* *`default=3000ms`* + Timeout for DB requests in milliseconds.
@@ -121,7 +118,7 @@ pipelines: type: string retry: 10 retention: 1s - increase_retention_exponentially: true + retention_exponentially_multiplier: 1.5 ``` input_example.json diff --git a/plugin/output/s3/README.md b/plugin/output/s3/README.md index f1ae1e386..c8bab7495 100755 --- a/plugin/output/s3/README.md +++ b/plugin/output/s3/README.md @@ -144,4 +144,23 @@ Sets upload timeout.
+**`retry`** *`uint64`* *`default=0`* + +Retries of upload. If File.d cannot upload for this number of attempts, +File.d will fall with non-zero exit code. + +
+ +**`retention`** *`cfg.Duration`* *`default=1s`* + +Retention milliseconds for retry to upload. + +
+ +**`retention_exponentially_multiplier`** *`float64`* *`default=1.5`* + +Multiplier for exponentially increase retention beetween retries + +
+
*Generated using [__insane-doc__](https://github.com/vitkovskii/insane-doc)* \ No newline at end of file diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index d9b412a5c..0c43f356c 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -238,7 +238,7 @@ type Config struct { // > @3@4@5@6 // > - // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > Retries of upload. If File.d cannot upload for this number of attempts, // > File.d will fall with non-zero exit code. Retry uint64 `json:"retry" default:"0"` // * @@ -532,7 +532,7 @@ func (p *Plugin) addFileJobWithBucket(bucketName string) func(filename string) { func (p *Plugin) uploadWork() { for compressed := range p.uploadCh { p.backoff.Reset() - backoff.Retry(func() error { + err := backoff.Retry(func() error { p.logger.Infof("starting upload s3 object. fileName=%s, bucketName=%s", compressed.fileName, compressed.bucketName) err := p.uploadToS3(compressed) if err == nil { @@ -548,6 +548,12 @@ func (p *Plugin) uploadWork() { p.logger.Errorf("could not upload object: %s, error: %s", compressed, err.Error()) return err }, p.backoff) + + if err != nil { + p.logger.Fatal("could not upload s3 object", zap.Error(err), + zap.Uint64("retries", p.config.Retry), + ) + } } } diff --git a/plugin/output/splunk/README.md b/plugin/output/splunk/README.md index d5583c5f7..fe1c0ef93 100755 --- a/plugin/output/splunk/README.md +++ b/plugin/output/splunk/README.md @@ -45,5 +45,24 @@ After this timeout the batch will be sent even if batch isn't completed.
+**`retry`** *`uint64`* *`default=0`* + +Retries of insertion. If File.d cannot insert for this number of attempts, +File.d will fall with non-zero exit code. + +
+ +**`retention`** *`cfg.Duration`* *`default=1s`* + +Retention milliseconds for retry to DB. + +
+ +**`retention_exponentially_multiplier`** *`float64`* *`default=1`* + +Multiplier for exponentially increase retention beetween retries + +
+
*Generated using [__insane-doc__](https://github.com/vitkovskii/insane-doc)* \ No newline at end of file diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index fdc1b4aba..f57ded1de 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -186,7 +186,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.logger.Debugf("trying to send: %s", outBuf) p.backoff.Reset() - backoff.Retry(func() error { + err := backoff.Retry(func() error { err := p.send(outBuf) if err != nil { p.sendErrorMetric.Inc() @@ -195,6 +195,12 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { return err }, p.backoff) + if err != nil { + p.logger.Fatal("can't send data to splunk", zap.Error(err), + zap.Uint64("retries", p.config.Retry), + ) + } + p.logger.Debugf("successfully sent: %s", outBuf) } From 2ac9858e6869c590ecdef3f770181c4c7a7fbd03 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Fri, 27 Oct 2023 18:50:29 +0700 Subject: [PATCH 08/20] backoff for gelf output --- plugin/output/gelf/README.md | 19 +++++++++++++++++ plugin/output/gelf/gelf.go | 40 ++++++++++++++++++++++++++++++++---- 2 files changed, 55 insertions(+), 4 deletions(-) diff --git a/plugin/output/gelf/README.md b/plugin/output/gelf/README.md index e3073a300..8ca7a048a 100755 --- a/plugin/output/gelf/README.md +++ b/plugin/output/gelf/README.md @@ -118,5 +118,24 @@ After this timeout the batch will be sent even if batch isn't completed.
+**`retry`** *`uint64`* *`default=0`* + +Retries of insertion. If File.d cannot insert for this number of attempts, +File.d will fall with non-zero exit code. + +
+ +**`retention`** *`cfg.Duration`* *`default=1s`* + +Retention milliseconds for retry to DB. + +
+ +**`retention_exponentially_multiplier`** *`float64`* *`default=1`* + +Multiplier for exponentially increase retention beetween retries + +
+
*Generated using [__insane-doc__](https://github.com/vitkovskii/insane-doc)* \ No newline at end of file diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 1abf1fdbd..65ca8cc8d 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -5,6 +5,7 @@ import ( "strings" "time" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -43,6 +44,7 @@ type Plugin struct { avgEventSize int batcher *pipeline.Batcher controller pipeline.OutputPluginController + backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -144,6 +146,23 @@ type Config struct { BatchFlushTimeout cfg.Duration `json:"batch_flush_timeout" default:"200ms" parse:"duration"` // * BatchFlushTimeout_ time.Duration + // > @3@4@5@6 + // > + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code. + Retry uint64 `json:"retry" default:"0"` // * + + // > @3@4@5@6 + // > + // > Retention milliseconds for retry to DB. + Retention cfg.Duration `json:"retention" default:"1s" parse:"duration"` // * + Retention_ time.Duration + + // > @3@4@5@6 + // > + // > Multiplier for exponentially increase retention beetween retries + RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + // fields converted to extra fields GELF format hostField string shortMessageField string @@ -178,6 +197,12 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config = config.(*Config) p.registerMetrics(params.MetricCtl) + p.backoff = cfg.GetBackoff( + p.config.Retention_, + p.config.RetentionExponentMultiplier, + p.config.Retry, + ) + p.config.hostField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.HostField)) p.config.shortMessageField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.ShortMessageField)) p.config.defaultShortMessageValue = strings.TrimSpace(p.config.DefaultShortMessageValue) @@ -245,7 +270,8 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = outBuf data.encodeBuf = encodeBuf - for { + p.backoff.Reset() + err := backoff.Retry(func() error { if data.gelf == nil { p.logger.Infof("connecting to gelf address=%s", p.config.Endpoint) @@ -254,7 +280,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.sendErrorMetric.Inc() p.logger.Errorf("can't connect to gelf endpoint address=%s: %s", p.config.Endpoint, err.Error()) time.Sleep(time.Second) - continue + return err } data.gelf = gelf } @@ -266,10 +292,16 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { _ = data.gelf.close() data.gelf = nil time.Sleep(time.Second) - continue + return err } - break + return nil + }, p.backoff) + + if err != nil { + p.logger.Fatal("can't send to gelf", zap.Error(err), + zap.Uint64("retries", p.config.Retry), + ) } } From 71d5916920a4a5b8cd7600274ec22f03f1542c7c Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Wed, 22 Nov 2023 13:45:26 +0700 Subject: [PATCH 09/20] skip_failed_insert in elasticsearch output --- plugin/output/clickhouse/README.md | 9 ++++++++- plugin/output/clickhouse/clickhouse.go | 17 ++++++++++++++-- plugin/output/elasticsearch/README.md | 9 ++++++++- plugin/output/elasticsearch/elasticsearch.go | 21 ++++++++++++++++---- 4 files changed, 48 insertions(+), 8 deletions(-) diff --git a/plugin/output/clickhouse/README.md b/plugin/output/clickhouse/README.md index 241e32617..62bcfe19f 100644 --- a/plugin/output/clickhouse/README.md +++ b/plugin/output/clickhouse/README.md @@ -111,7 +111,14 @@ If the strict mode is enabled file.d fails (exit with code 1) in above examples. **`retry`** *`uint64`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see skip_failed_insert). + +
+ +**`skip_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or skip the message +**Experimental feature**
diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index 25feeac6a..f5a99a97f 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -229,9 +229,15 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code. + // > File.d will fall with non-zero exit code or skip message (see skip_failed_insert). Retry uint64 `json:"retry" default:"10"` // * + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or skip the message + // > **Experimental feature** + SkipFailedInsert bool `json:"skip_failed_insert" default:"false"` // * + // > @3@4@5@6 // > // > Additional settings to the Clickhouse. @@ -515,7 +521,14 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.logger.Fatal("can't insert to the table", zap.Error(err), + var errLogFunc func(msg string, fields ...zap.Field) + if p.config.SkipFailedInsert { + errLogFunc = p.logger.Error + } else { + errLogFunc = p.logger.Fatal + } + + errLogFunc("can't insert to the table", zap.Error(err), zap.Uint64("retries", p.config.Retry), zap.String("table", p.config.Table)) } diff --git a/plugin/output/elasticsearch/README.md b/plugin/output/elasticsearch/README.md index a263d8442..09a06b37b 100755 --- a/plugin/output/elasticsearch/README.md +++ b/plugin/output/elasticsearch/README.md @@ -97,7 +97,14 @@ Operation type to be used in batch requests. It can be `index` or `create`. Defa **`retry`** *`uint64`* *`default=0`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see skip_failed_insert). + +
+ +**`skip_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or skip the message +**Experimental feature**
diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 5c6047d56..82884bbe0 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -144,9 +144,15 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code. + // > File.d will fall with non-zero exit code or skip message (see skip_failed_insert). Retry uint64 `json:"retry" default:"0"` // * + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or skip the message + // > **Experimental feature** + SkipFailedInsert bool `json:"skip_failed_insert" default:"false"` // * + // > @3@4@5@6 // > // > Retention milliseconds for retry to DB. @@ -281,8 +287,8 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.backoff.Reset() err := backoff.Retry(func() error { - var err error - if err := p.send(data.outBuf); err != nil { + err := p.send(data.outBuf) + if err != nil { p.sendErrorMetric.Inc() p.logger.Error("can't send to the elastic, will try other endpoint", zap.Error(err)) } @@ -290,7 +296,14 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.logger.Fatal("can't send to the elastic", zap.Error(err), + var errLogFunc func(args ...interface{}) + if p.config.SkipFailedInsert { + errLogFunc = p.logger.Sugar().Error + } else { + errLogFunc = p.logger.Sugar().Fatal + } + + errLogFunc("can't send to the elastic", zap.Error(err), zap.Uint64("retries", p.config.Retry), ) } From 3e1cf43430851671c538ebe9934170426f8d3b85 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Thu, 23 Nov 2023 16:38:57 +0700 Subject: [PATCH 10/20] use fatal_on_failed_insert --- go.mod | 2 ++ plugin/README.md | 6 ++-- plugin/output/README.md | 6 ++-- plugin/output/clickhouse/README.md | 10 +++--- plugin/output/clickhouse/clickhouse.go | 25 +++++++-------- plugin/output/elasticsearch/README.md | 10 +++--- plugin/output/elasticsearch/elasticsearch.go | 22 ++++++------- plugin/output/gelf/README.md | 13 ++++++-- plugin/output/gelf/gelf.go | 27 +++++++++++----- plugin/output/kafka/README.md | 13 ++++++-- plugin/output/kafka/kafka.go | 27 +++++++++++----- plugin/output/postgres/README.md | 12 +++++-- plugin/output/postgres/postgres.go | 30 +++++++++++++----- plugin/output/postgres/postgres_test.go | 7 +++++ plugin/output/s3/README.md | 19 +++++++---- plugin/output/s3/s3.go | 33 ++++++++++++++------ plugin/output/splunk/README.md | 13 ++++++-- plugin/output/splunk/splunk.go | 27 +++++++++++----- plugin/output/splunk/splunk_test.go | 5 ++- 19 files changed, 209 insertions(+), 98 deletions(-) diff --git a/go.mod b/go.mod index 87ec647af..68299446f 100644 --- a/go.mod +++ b/go.mod @@ -14,6 +14,7 @@ require ( github.com/alicebob/miniredis/v2 v2.30.5 github.com/bitly/go-simplejson v0.5.1 github.com/cespare/xxhash/v2 v2.2.0 + github.com/cenkalti/backoff/v3 v3.0.0 github.com/euank/go-kmsg-parser v2.0.0+incompatible github.com/go-redis/redis v6.15.9+incompatible github.com/golang/mock v1.6.0 @@ -49,6 +50,7 @@ require ( github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v3 v3.0.0 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cilium/ebpf v0.9.1 // indirect github.com/containerd/cgroups/v3 v3.0.1 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect diff --git a/plugin/README.md b/plugin/README.md index c9a21f163..822431a6f 100755 --- a/plugin/README.md +++ b/plugin/README.md @@ -733,9 +733,9 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 file_config: diff --git a/plugin/output/README.md b/plugin/output/README.md index 124405eac..9b05cc4ef 100755 --- a/plugin/output/README.md +++ b/plugin/output/README.md @@ -72,9 +72,9 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 file_config: diff --git a/plugin/output/clickhouse/README.md b/plugin/output/clickhouse/README.md index 62bcfe19f..43140deae 100644 --- a/plugin/output/clickhouse/README.md +++ b/plugin/output/clickhouse/README.md @@ -108,16 +108,16 @@ If the strict mode is enabled file.d fails (exit with code 1) in above examples.
-**`retry`** *`uint64`* *`default=10`* +**`retry`** *`int`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code or skip message (see skip_failed_insert). +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert).
-**`skip_failed_insert`** *`bool`* *`default=false`* +**`fatal_on_failed_insert`** *`bool`* *`default=false`* -After an insert error, fall with a non-zero exit code or skip the message +After an insert error, fall with a non-zero exit code or not **Experimental feature**
@@ -135,7 +135,7 @@ Retention milliseconds for retry to DB.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index f5a99a97f..f66053bd0 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -229,14 +229,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code or skip message (see skip_failed_insert). - Retry uint64 `json:"retry" default:"10"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * // > @3@4@5@6 // > - // > After an insert error, fall with a non-zero exit code or skip the message + // > After an insert error, fall with a non-zero exit code or not // > **Experimental feature** - SkipFailedInsert bool `json:"skip_failed_insert" default:"false"` // * + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -253,7 +253,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // > @3@4@5@6 // > @@ -341,9 +341,6 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.registerMetrics(params.MetricCtl) p.ctx, p.cancelFunc = context.WithCancel(context.Background()) - if p.config.Retry < 1 { - p.logger.Fatal("'retry' can't be <1") - } if p.config.Retention_ < 1 { p.logger.Fatal("'retention' can't be <1") } @@ -420,8 +417,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ @@ -522,14 +519,14 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { if err != nil { var errLogFunc func(msg string, fields ...zap.Field) - if p.config.SkipFailedInsert { - errLogFunc = p.logger.Error - } else { + if p.config.FatalOnFailedInsert { errLogFunc = p.logger.Fatal + } else { + errLogFunc = p.logger.Error } errLogFunc("can't insert to the table", zap.Error(err), - zap.Uint64("retries", p.config.Retry), + zap.Int("retries", p.config.Retry), zap.String("table", p.config.Table)) } } diff --git a/plugin/output/elasticsearch/README.md b/plugin/output/elasticsearch/README.md index 09a06b37b..c92cb9506 100755 --- a/plugin/output/elasticsearch/README.md +++ b/plugin/output/elasticsearch/README.md @@ -94,16 +94,16 @@ Operation type to be used in batch requests. It can be `index` or `create`. Defa
-**`retry`** *`uint64`* *`default=0`* +**`retry`** *`int`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code or skip message (see skip_failed_insert). +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert).
-**`skip_failed_insert`** *`bool`* *`default=false`* +**`fatal_on_failed_insert`** *`bool`* *`default=false`* -After an insert error, fall with a non-zero exit code or skip the message +After an insert error, fall with a non-zero exit code or not **Experimental feature**
@@ -114,7 +114,7 @@ Retention milliseconds for retry to DB.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 82884bbe0..f4fc6807e 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -144,14 +144,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code or skip message (see skip_failed_insert). - Retry uint64 `json:"retry" default:"0"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * // > @3@4@5@6 // > - // > After an insert error, fall with a non-zero exit code or skip the message + // > After an insert error, fall with a non-zero exit code or not // > **Experimental feature** - SkipFailedInsert bool `json:"skip_failed_insert" default:"false"` // * + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -162,7 +162,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } type data struct { @@ -246,8 +246,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) p.batcher.Start(ctx) @@ -297,14 +297,14 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { if err != nil { var errLogFunc func(args ...interface{}) - if p.config.SkipFailedInsert { - errLogFunc = p.logger.Sugar().Error - } else { + if p.config.FatalOnFailedInsert { errLogFunc = p.logger.Sugar().Fatal + } else { + errLogFunc = p.logger.Sugar().Error } errLogFunc("can't send to the elastic", zap.Error(err), - zap.Uint64("retries", p.config.Retry), + zap.Int("retries", p.config.Retry), ) } } diff --git a/plugin/output/gelf/README.md b/plugin/output/gelf/README.md index 8ca7a048a..9ec4fcfc4 100755 --- a/plugin/output/gelf/README.md +++ b/plugin/output/gelf/README.md @@ -118,10 +118,17 @@ After this timeout the batch will be sent even if batch isn't completed.
-**`retry`** *`uint64`* *`default=0`* +**`retry`** *`int`* *`default=0`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + +
+ +**`fatal_on_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or not +**Experimental feature**
@@ -131,7 +138,7 @@ Retention milliseconds for retry to DB.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 65ca8cc8d..9b5643588 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -149,8 +149,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code. - Retry uint64 `json:"retry" default:"0"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"0"` // * + + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or not + // > **Experimental feature** + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -161,7 +167,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // fields converted to extra fields GELF format hostField string @@ -199,8 +205,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) p.config.hostField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.HostField)) @@ -299,8 +305,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.logger.Fatal("can't send to gelf", zap.Error(err), - zap.Uint64("retries", p.config.Retry), + var errLogFunc func(args ...interface{}) + if p.config.FatalOnFailedInsert { + errLogFunc = p.logger.Fatal + } else { + errLogFunc = p.logger.Error + } + + errLogFunc("can't send to gelf", zap.Error(err), + zap.Int("retries", p.config.Retry), ) } } diff --git a/plugin/output/kafka/README.md b/plugin/output/kafka/README.md index cc77cd772..c9394e4bc 100755 --- a/plugin/output/kafka/README.md +++ b/plugin/output/kafka/README.md @@ -57,10 +57,17 @@ After this timeout the batch will be sent even if batch isn't full.
-**`retry`** *`uint64`* *`default=10`* +**`retry`** *`int`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + +
+ +**`fatal_on_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or not +**Experimental feature**
@@ -70,7 +77,7 @@ Retention milliseconds for retry.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index 9b98f0553..39c1b9d93 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -100,8 +100,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code. - Retry uint64 `json:"retry" default:"10"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * + + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or not + // > **Experimental feature** + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -112,7 +118,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // > @3@4@5@6 // > @@ -173,8 +179,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP } p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) p.logger.Infof("workers count=%d, batch size=%d", p.config.WorkersCount_, p.config.BatchSize_) @@ -262,8 +268,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.logger.Error("can't write batch", zap.Error(err), - zap.Uint64("retries", p.config.Retry)) + var errLogFunc func(args ...interface{}) + if p.config.FatalOnFailedInsert { + errLogFunc = p.logger.Fatal + } else { + errLogFunc = p.logger.Error + } + + errLogFunc("can't write batch", zap.Error(err), + zap.Int("retries", p.config.Retry)) } } diff --git a/plugin/output/postgres/README.md b/plugin/output/postgres/README.md index 3b30c4b84..40c2b49b8 100755 --- a/plugin/output/postgres/README.md +++ b/plugin/output/postgres/README.md @@ -38,9 +38,17 @@ and nullable options.
-**`retry`** *`uint64`* *`default=3`* +**`retry`** *`int`* *`default=10`* -Retries of insertion. +Retries of insertion. If File.d cannot insert for this number of attempts, +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + +
+ +**`fatal_on_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or not +**Experimental feature**
diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index e7a6e2bbc..b48d7f923 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -158,8 +158,15 @@ type Config struct { // > @3@4@5@6 // > - // > Retries of insertion. - Retry uint64 `json:"retry" default:"3"` // * + // > Retries of insertion. If File.d cannot insert for this number of attempts, + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * + + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or not + // > **Experimental feature** + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -170,7 +177,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // > @3@4@5@6 // > @@ -284,8 +291,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) ctx, cancel := context.WithCancel(context.Background()) @@ -367,7 +374,7 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { p.backoff.Reset() err = backoff.Retry(func() error { - err = p.try(query, argsSliceInterface) + err := p.try(query, argsSliceInterface) if err != nil { p.insertErrorsMetric.Inc() p.logger.Errorf("can't exec query: %s", err.Error()) @@ -378,8 +385,15 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.pool.Close() - p.logger.Fatalf("failed insert into %s. query: %s, args: %v, err: %v", p.config.Table, query, args, err) + var errLogFunc func(args ...interface{}) + if p.config.FatalOnFailedInsert { + errLogFunc = p.logger.Fatal + p.pool.Close() + } else { + errLogFunc = p.logger.Error + } + + errLogFunc("failed insert into %s. query: %s, args: %v, err: %v", p.config.Table, query, args, err) } } diff --git a/plugin/output/postgres/postgres_test.go b/plugin/output/postgres/postgres_test.go index 522c874be..a8e76a9ce 100644 --- a/plugin/output/postgres/postgres_test.go +++ b/plugin/output/postgres/postgres_test.go @@ -10,6 +10,7 @@ import ( "github.com/golang/mock/gomock" "github.com/jackc/pgconn" "github.com/jackc/pgproto3/v2" + "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/ozontech/file.d/pipeline" @@ -87,6 +88,7 @@ func TestPrivateOut(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } @@ -162,6 +164,7 @@ func TestPrivateOutWithRetry(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } @@ -216,6 +219,7 @@ func TestPrivateOutNoGoodEvents(t *testing.T) { config: &config, queryBuilder: builder, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) @@ -295,6 +299,7 @@ func TestPrivateOutDeduplicatedEvents(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } @@ -362,6 +367,7 @@ func TestPrivateOutWrongTypeInField(t *testing.T) { config: &config, queryBuilder: builder, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) @@ -466,6 +472,7 @@ func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *te queryBuilder: builder, pool: pool, logger: testLogger, + backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } diff --git a/plugin/output/s3/README.md b/plugin/output/s3/README.md index c8bab7495..d122b0500 100755 --- a/plugin/output/s3/README.md +++ b/plugin/output/s3/README.md @@ -22,9 +22,9 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 file_config: @@ -144,10 +144,17 @@ Sets upload timeout.
-**`retry`** *`uint64`* *`default=0`* +**`retry`** *`int`* *`default=10`* Retries of upload. If File.d cannot upload for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + +
+ +**`fatal_on_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or not +**Experimental feature**
@@ -157,7 +164,7 @@ Retention milliseconds for retry to upload.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1.5`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index 0c43f356c..7407bdb94 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -48,9 +48,9 @@ pipelines: type: http emulate_mode: "no" address: ":9200" - actions: - - type: json_decode - field: message + actions: + - type: json_decode + field: message output: type: s3 file_config: @@ -239,8 +239,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of upload. If File.d cannot upload for this number of attempts, - // > File.d will fall with non-zero exit code. - Retry uint64 `json:"retry" default:"0"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * + + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or not + // > **Experimental feature** + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -251,7 +257,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1.5"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } func (c *Config) IsMultiBucketExists(bucketName string) bool { @@ -299,8 +305,8 @@ func (p *Plugin) StartWithMinio(config pipeline.AnyConfig, params *pipeline.Outp p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) // outPlugCount is defaultBucket + multi_buckets count, use to set maps size. @@ -550,8 +556,15 @@ func (p *Plugin) uploadWork() { }, p.backoff) if err != nil { - p.logger.Fatal("could not upload s3 object", zap.Error(err), - zap.Uint64("retries", p.config.Retry), + var errLogFunc func(args ...interface{}) + if p.config.FatalOnFailedInsert { + errLogFunc = p.logger.Fatal + } else { + errLogFunc = p.logger.Error + } + + errLogFunc("could not upload s3 object", zap.Error(err), + zap.Int("retries", p.config.Retry), ) } } diff --git a/plugin/output/splunk/README.md b/plugin/output/splunk/README.md index fe1c0ef93..e6be14f53 100755 --- a/plugin/output/splunk/README.md +++ b/plugin/output/splunk/README.md @@ -45,10 +45,17 @@ After this timeout the batch will be sent even if batch isn't completed.
-**`retry`** *`uint64`* *`default=0`* +**`retry`** *`int`* *`default=10`* Retries of insertion. If File.d cannot insert for this number of attempts, -File.d will fall with non-zero exit code. +File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + +
+ +**`fatal_on_failed_insert`** *`bool`* *`default=false`* + +After an insert error, fall with a non-zero exit code or not +**Experimental feature**
@@ -58,7 +65,7 @@ Retention milliseconds for retry to DB.
-**`retention_exponentially_multiplier`** *`float64`* *`default=1`* +**`retention_exponentially_multiplier`** *`int`* *`default=2`* Multiplier for exponentially increase retention beetween retries diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index f57ded1de..d711b9eb2 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -88,8 +88,14 @@ type Config struct { // > @3@4@5@6 // > // > Retries of insertion. If File.d cannot insert for this number of attempts, - // > File.d will fall with non-zero exit code. - Retry uint64 `json:"retry" default:"0"` // * + // > File.d will fall with non-zero exit code or skip message (see fatal_on_failed_insert). + Retry int `json:"retry" default:"10"` // * + + // > @3@4@5@6 + // > + // > After an insert error, fall with a non-zero exit code or not + // > **Experimental feature** + FatalOnFailedInsert bool `json:"fatal_on_failed_insert" default:"false"` // * // > @3@4@5@6 // > @@ -100,7 +106,7 @@ type Config struct { // > @3@4@5@6 // > // > Multiplier for exponentially increase retention beetween retries - RetentionExponentMultiplier float64 `json:"retention_exponentially_multiplier" default:"1"` // * + RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } type data struct { @@ -127,8 +133,8 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.client = p.newClient(p.config.RequestTimeout_) p.backoff = cfg.GetBackoff( p.config.Retention_, - p.config.RetentionExponentMultiplier, - p.config.Retry, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), ) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ @@ -196,8 +202,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { }, p.backoff) if err != nil { - p.logger.Fatal("can't send data to splunk", zap.Error(err), - zap.Uint64("retries", p.config.Retry), + var errLogFunc func(args ...interface{}) + if p.config.FatalOnFailedInsert { + errLogFunc = p.logger.Fatal + } else { + errLogFunc = p.logger.Error + } + + errLogFunc("can't send data to splunk", zap.Error(err), + zap.Int("retries", p.config.Retry), ) } diff --git a/plugin/output/splunk/splunk_test.go b/plugin/output/splunk/splunk_test.go index 2661ecf48..472bfeebe 100644 --- a/plugin/output/splunk/splunk_test.go +++ b/plugin/output/splunk/splunk_test.go @@ -5,7 +5,9 @@ import ( "net/http" "net/http/httptest" "testing" + "time" + "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/pipeline" "github.com/stretchr/testify/assert" insaneJSON "github.com/vitkovskii/insane-json" @@ -48,7 +50,8 @@ func TestSplunk(t *testing.T) { config: &Config{ Endpoint: testServer.URL, }, - logger: zap.NewExample().Sugar(), + backoff: cfg.GetBackoff(1*time.Second, 1, 2), + logger: zap.NewExample().Sugar(), } batch := pipeline.NewPreparedBatch([]*pipeline.Event{ From adad6725cb140fcdfe0a8418caaacd801c65410c Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Wed, 29 Nov 2023 18:36:37 +0700 Subject: [PATCH 11/20] s3: backoff for every worker --- plugin/output/s3/s3.go | 20 ++++++++------------ 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index 7407bdb94..ce32d3303 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -148,8 +148,6 @@ type Plugin struct { compressor compressor - backoff backoff.BackOff - // plugin metrics sendErrorMetric prometheus.Counter uploadFileMetric *prometheus.CounterVec @@ -303,12 +301,6 @@ func (p *Plugin) StartWithMinio(config pipeline.AnyConfig, params *pipeline.Outp p.config = config.(*Config) p.params = params - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) - // outPlugCount is defaultBucket + multi_buckets count, use to set maps size. outPlugCount := len(p.config.MultiBuckets) + 1 p.limiter = NewObjectStoreClientLimiter(p.config.DynamicBucketsLimit + outPlugCount) @@ -343,7 +335,11 @@ func (p *Plugin) StartWithMinio(config pipeline.AnyConfig, params *pipeline.Outp p.compressCh = make(chan fileDTO, p.config.FileConfig.WorkersCount_) for i := 0; i < p.config.FileConfig.WorkersCount_; i++ { - go p.uploadWork() + go p.uploadWork(cfg.GetBackoff( + p.config.Retention_, + float64(p.config.RetentionExponentMultiplier), + uint64(p.config.Retry), + )) go p.compressWork() } err = p.startPlugins(params, outPlugCount, targetDirs, fileNames) @@ -535,9 +531,9 @@ func (p *Plugin) addFileJobWithBucket(bucketName string) func(filename string) { } } -func (p *Plugin) uploadWork() { +func (p *Plugin) uploadWork(workerBackoff backoff.BackOff) { for compressed := range p.uploadCh { - p.backoff.Reset() + workerBackoff.Reset() err := backoff.Retry(func() error { p.logger.Infof("starting upload s3 object. fileName=%s, bucketName=%s", compressed.fileName, compressed.bucketName) err := p.uploadToS3(compressed) @@ -553,7 +549,7 @@ func (p *Plugin) uploadWork() { } p.logger.Errorf("could not upload object: %s, error: %s", compressed, err.Error()) return err - }, p.backoff) + }, workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) From 6fc84516112f114a28d71f8d123cbbc8f3801724 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Fri, 8 Dec 2023 13:41:44 +0700 Subject: [PATCH 12/20] create backoff for every batch worker --- pipeline/batch.go | 38 +++++++++++++------- pipeline/batch_test.go | 5 +-- plugin/output/clickhouse/clickhouse.go | 34 ++++++++---------- plugin/output/elasticsearch/elasticsearch.go | 37 +++++++++---------- plugin/output/file/file.go | 3 +- plugin/output/gelf/gelf.go | 38 +++++++++----------- plugin/output/kafka/kafka.go | 33 ++++++++--------- plugin/output/kafka/kafka_test.go | 5 ++- plugin/output/postgres/postgres.go | 35 ++++++++---------- plugin/output/postgres/postgres_test.go | 24 ++++++------- plugin/output/splunk/splunk.go | 35 +++++++++--------- plugin/output/splunk/splunk_test.go | 7 ++-- 12 files changed, 144 insertions(+), 150 deletions(-) diff --git a/pipeline/batch.go b/pipeline/batch.go index 4fb0380fa..ba4cf2428 100644 --- a/pipeline/batch.go +++ b/pipeline/batch.go @@ -5,6 +5,8 @@ import ( "sync" "time" + "github.com/cenkalti/backoff/v3" + "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/prometheus/client_golang/prometheus" @@ -133,21 +135,24 @@ type Batcher struct { } type ( - BatcherOutFn func(*WorkerData, *Batch) + BatcherOutFn func(*WorkerData, *Batch, *backoff.BackOff) BatcherMaintenanceFn func(*WorkerData) BatcherOptions struct { - PipelineName string - OutputType string - OutFn BatcherOutFn - MaintenanceFn BatcherMaintenanceFn - Controller OutputPluginController - Workers int - BatchSizeCount int - BatchSizeBytes int - FlushTimeout time.Duration - MaintenanceInterval time.Duration - MetricCtl *metric.Ctl + PipelineName string + OutputType string + OutFn BatcherOutFn + MaintenanceFn BatcherMaintenanceFn + Controller OutputPluginController + Workers int + BatchSizeCount int + BatchSizeBytes int + FlushTimeout time.Duration + MaintenanceInterval time.Duration + MetricCtl *metric.Ctl + Retry int + RetryRetention time.Duration + RetryRetentionExponentMultiplier int } ) @@ -197,12 +202,19 @@ func (b *Batcher) work() { t := time.Now() data := WorkerData(nil) + workerBackoff := cfg.GetBackoff( + b.opts.RetryRetention, + float64(b.opts.RetryRetentionExponentMultiplier), + uint64(b.opts.Retry), + ) + for batch := range b.fullBatches { b.workersInProgress.Inc() if batch.hasIterableEvents { now := time.Now() - b.opts.OutFn(&data, batch) + workerBackoff.Reset() + b.opts.OutFn(&data, batch, &workerBackoff) b.batchOutFnSeconds.Observe(time.Since(now).Seconds()) } diff --git a/pipeline/batch_test.go b/pipeline/batch_test.go index 8ae5fae57..c00df3c75 100644 --- a/pipeline/batch_test.go +++ b/pipeline/batch_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/prometheus/client_golang/prometheus" @@ -36,7 +37,7 @@ func TestBatcher(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch) { + batcherOut := func(workerData *WorkerData, batch *Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = batchCount } @@ -107,7 +108,7 @@ func TestBatcherMaxSize(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch) { + batcherOut := func(workerData *WorkerData, batch *Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = batchCount } diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index f66053bd0..1d16a50f0 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -47,7 +47,6 @@ type Plugin struct { batcher *pipeline.Batcher ctx context.Context cancelFunc context.CancelFunc - backoff backoff.BackOff query string @@ -415,22 +414,19 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP } } - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: params.Controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + Controller: params.Controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) p.batcher.Start(p.ctx) @@ -459,7 +455,7 @@ func (d data) reset() { } } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { // we don't check the error, schema already validated in the Start columns, _ := inferInsaneColInputs(p.config.Columns) @@ -499,8 +495,8 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } }) - p.backoff.Reset() try := 0 + (*workerBackoff).Reset() err := backoff.Retry(func() error { requestID := p.requestID.Inc() clickhouse := p.getInstance(requestID, try) @@ -515,7 +511,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { ) } return err - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(msg string, fields ...zap.Field) diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index f4fc6807e..b90aa898b 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -49,7 +49,6 @@ type Plugin struct { batcher *pipeline.Batcher controller pipeline.OutputPluginController mu *sync.Mutex - backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -228,28 +227,25 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.logger.Info("starting batcher", zap.Duration("timeout", p.config.BatchFlushTimeout_)) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MaintenanceInterval: time.Minute, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MaintenanceInterval: time.Minute, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) ctx, cancel := context.WithCancel(context.Background()) p.cancel = cancel - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) - p.batcher.Start(ctx) } @@ -267,7 +263,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.indexingErrorsMetric = ctl.RegisterCounter("output_elasticsearch_index_error", "Number of elasticsearch indexing errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -285,7 +281,6 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = p.appendEvent(data.outBuf, event) }) - p.backoff.Reset() err := backoff.Retry(func() error { err := p.send(data.outBuf) if err != nil { @@ -293,7 +288,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.logger.Error("can't send to the elastic, will try other endpoint", zap.Error(err)) } return err - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) diff --git a/plugin/output/file/file.go b/plugin/output/file/file.go index 8311e08da..8761b1900 100644 --- a/plugin/output/file/file.go +++ b/plugin/output/file/file.go @@ -9,6 +9,7 @@ import ( "sync" "time" + "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/logger" @@ -181,7 +182,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 9b5643588..6eb068702 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -44,7 +44,6 @@ type Plugin struct { avgEventSize int batcher *pipeline.Batcher controller pipeline.OutputPluginController - backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -203,12 +202,6 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config = config.(*Config) p.registerMetrics(params.MetricCtl) - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) - p.config.hostField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.HostField)) p.config.shortMessageField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.ShortMessageField)) p.config.defaultShortMessageValue = strings.TrimSpace(p.config.DefaultShortMessageValue) @@ -222,17 +215,20 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config.levelField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.LevelField)) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MaintenanceInterval: p.config.ReconnectInterval_, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MaintenanceInterval: p.config.ReconnectInterval_, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) p.batcher.Start(context.TODO()) @@ -250,7 +246,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.sendErrorMetric = ctl.RegisterCounter("output_gelf_send_error", "Total GELF send errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -276,7 +272,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = outBuf data.encodeBuf = encodeBuf - p.backoff.Reset() + (*workerBackoff).Reset() err := backoff.Retry(func() error { if data.gelf == nil { p.logger.Infof("connecting to gelf address=%s", p.config.Endpoint) @@ -302,7 +298,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { } return nil - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index 39c1b9d93..38bc76696 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -38,7 +38,6 @@ type Plugin struct { producer sarama.SyncProducer batcher *pipeline.Batcher - backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -177,25 +176,23 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP if p.config.Retention_ < 1 { p.logger.Fatal("'retention' can't be <1") } - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) p.logger.Infof("workers count=%d, batch size=%d", p.config.WorkersCount_, p.config.BatchSize_) p.producer = NewProducer(p.config, p.logger) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) p.batcher.Start(context.TODO()) @@ -209,7 +206,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.sendErrorMetric = ctl.RegisterCounter("output_kafka_send_errors", "Total Kafka send errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = &data{ messages: make([]*sarama.ProducerMessage, p.config.BatchSize_), @@ -247,7 +244,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { data.outBuf = outBuf - p.backoff.Reset() + (*workerBackoff).Reset() err := backoff.Retry(func() error { err := p.producer.SendMessages(data.messages[:i]) if err == nil { @@ -265,7 +262,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { ) return err - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) diff --git a/plugin/output/kafka/kafka_test.go b/plugin/output/kafka/kafka_test.go index 552a494a3..748b19e10 100644 --- a/plugin/output/kafka/kafka_test.go +++ b/plugin/output/kafka/kafka_test.go @@ -6,8 +6,10 @@ package kafka import ( "fmt" "testing" + "time" "github.com/Shopify/sarama" + "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/pipeline" "github.com/stretchr/testify/require" insaneJSON "github.com/vitkovskii/insane-json" @@ -116,6 +118,7 @@ func FuzzKafka(f *testing.F) { producer: nil, batcher: nil, } + backoff := cfg.GetBackoff(1*time.Second, 1, 2) f.Fuzz(func(t *testing.T, topicField, topicVal, key, val string) { p.producer = &mockProducer{ @@ -125,6 +128,6 @@ func FuzzKafka(f *testing.F) { data := pipeline.NewPreparedBatch([]*pipeline.Event{ newEvent(t, topicField, topicVal, key, val), }) - p.out(&worker, data) + p.out(&worker, data, &backoff) }) } diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index b48d7f923..80b260ef1 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -105,8 +105,6 @@ type Plugin struct { queryBuilder PgQueryBuilder pool PgxIface - backoff backoff.BackOff - // plugin metrics discardedEventMetric prometheus.Counter @@ -278,23 +276,20 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.pool = pool p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) - ctx, cancel := context.WithCancel(context.Background()) p.ctx = ctx p.cancelFunc = cancel @@ -312,7 +307,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { // _ *pipeline.WorkerData - doesn't required in this plugin, we can't parse // events for uniques through bytes. builder := p.queryBuilder.GetInsertBuilder() @@ -372,7 +367,7 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { argsSliceInterface[i] = args[i-1] } - p.backoff.Reset() + (*workerBackoff).Reset() err = backoff.Retry(func() error { err := p.try(query, argsSliceInterface) if err != nil { @@ -382,7 +377,7 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) { } p.writtenEventMetric.Add(float64(len(uniqueEventsMap))) return nil - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) diff --git a/plugin/output/postgres/postgres_test.go b/plugin/output/postgres/postgres_test.go index a8e76a9ce..859a2e701 100644 --- a/plugin/output/postgres/postgres_test.go +++ b/plugin/output/postgres/postgres_test.go @@ -88,14 +88,14 @@ func TestPrivateOut(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } func TestPrivateOutWithRetry(t *testing.T) { @@ -164,14 +164,14 @@ func TestPrivateOutWithRetry(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } func TestPrivateOutNoGoodEvents(t *testing.T) { @@ -219,13 +219,13 @@ func TestPrivateOutNoGoodEvents(t *testing.T) { config: &config, queryBuilder: builder, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } func TestPrivateOutDeduplicatedEvents(t *testing.T) { @@ -299,7 +299,6 @@ func TestPrivateOutDeduplicatedEvents(t *testing.T) { queryBuilder: builder, pool: pool, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } @@ -310,7 +309,8 @@ func TestPrivateOutDeduplicatedEvents(t *testing.T) { {Root: rootDuplication}, {Root: rootDuplicationMore}, }) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } func TestPrivateOutWrongTypeInField(t *testing.T) { @@ -367,13 +367,13 @@ func TestPrivateOutWrongTypeInField(t *testing.T) { config: &config, queryBuilder: builder, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), } p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *testing.T) { @@ -472,7 +472,6 @@ func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *te queryBuilder: builder, pool: pool, logger: testLogger, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), ctx: ctx, } @@ -485,7 +484,8 @@ func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *te {Root: secondUniqueRoot}, {Root: badRoot}, }) - p.out(nil, batch) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + p.out(nil, batch, &backoff) } // TODO replace with gomock diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index d711b9eb2..c99abdfb1 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -35,7 +35,6 @@ type Plugin struct { avgEventSize int batcher *pipeline.Batcher controller pipeline.OutputPluginController - backoff backoff.BackOff // plugin metrics sendErrorMetric prometheus.Counter @@ -131,23 +130,21 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config = config.(*Config) p.registerMetrics(params.MetricCtl) p.client = p.newClient(p.config.RequestTimeout_) - p.backoff = cfg.GetBackoff( - p.config.Retention_, - float64(p.config.RetentionExponentMultiplier), - uint64(p.config.Retry), - ) p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, + PipelineName: params.PipelineName, + OutputType: outPluginType, + OutFn: p.out, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + Retry: p.config.Retry, + RetryRetention: p.config.Retention_, + RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, }) p.batcher.Start(context.TODO()) @@ -165,7 +162,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -191,7 +188,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.logger.Debugf("trying to send: %s", outBuf) - p.backoff.Reset() + (*workerBackoff).Reset() err := backoff.Retry(func() error { err := p.send(outBuf) if err != nil { @@ -199,7 +196,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { p.logger.Errorf("can't send data to splunk address=%s: %s", p.config.Endpoint, err.Error()) } return err - }, p.backoff) + }, *workerBackoff) if err != nil { var errLogFunc func(args ...interface{}) diff --git a/plugin/output/splunk/splunk_test.go b/plugin/output/splunk/splunk_test.go index 472bfeebe..7258b0ed5 100644 --- a/plugin/output/splunk/splunk_test.go +++ b/plugin/output/splunk/splunk_test.go @@ -50,8 +50,7 @@ func TestSplunk(t *testing.T) { config: &Config{ Endpoint: testServer.URL, }, - backoff: cfg.GetBackoff(1*time.Second, 1, 2), - logger: zap.NewExample().Sugar(), + logger: zap.NewExample().Sugar(), } batch := pipeline.NewPreparedBatch([]*pipeline.Event{ @@ -59,8 +58,10 @@ func TestSplunk(t *testing.T) { {Root: input}, }) + backoff := cfg.GetBackoff(1*time.Second, 1, 2) + data := pipeline.WorkerData(nil) - plugin.out(&data, batch) + plugin.out(&data, batch, &backoff) assert.Equal(t, testCase.expected+testCase.expected, string(response)) }) From 29306ae95ebff7c9e7a82fb5af06d20220b648c6 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Thu, 21 Dec 2023 16:56:05 +0700 Subject: [PATCH 13/20] fix errLogFunc type --- plugin/output/elasticsearch/elasticsearch.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index b90aa898b..eeb488e13 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -291,7 +291,7 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor }, *workerBackoff) if err != nil { - var errLogFunc func(args ...interface{}) + var errLogFunc func(msg string, fields ...zap.Field) if p.config.FatalOnFailedInsert { errLogFunc = p.logger.Sugar().Fatal } else { From b1cce1320945079f48da1360eaa2baaca821849c Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Mon, 15 Jan 2024 19:10:40 +0700 Subject: [PATCH 14/20] Update plugin/output/clickhouse/clickhouse.go Co-authored-by: Oleg Don <31502412+HeadHunter483@users.noreply.github.com> --- plugin/output/clickhouse/README.md | 2 +- plugin/output/clickhouse/clickhouse.go | 2 +- plugin/output/elasticsearch/README.md | 2 +- plugin/output/elasticsearch/elasticsearch.go | 2 +- plugin/output/gelf/README.md | 2 +- plugin/output/gelf/gelf.go | 2 +- plugin/output/kafka/README.md | 2 +- plugin/output/kafka/kafka.go | 2 +- plugin/output/postgres/README.md | 2 +- plugin/output/postgres/postgres.go | 2 +- plugin/output/s3/README.md | 2 +- plugin/output/s3/s3.go | 2 +- plugin/output/splunk/README.md | 2 +- plugin/output/splunk/splunk.go | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/plugin/output/clickhouse/README.md b/plugin/output/clickhouse/README.md index 43140deae..a0a7ce227 100644 --- a/plugin/output/clickhouse/README.md +++ b/plugin/output/clickhouse/README.md @@ -137,7 +137,7 @@ Retention milliseconds for retry to DB. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index 1d16a50f0..e54b01f6e 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -251,7 +251,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // > @3@4@5@6 diff --git a/plugin/output/elasticsearch/README.md b/plugin/output/elasticsearch/README.md index c92cb9506..b8587e752 100755 --- a/plugin/output/elasticsearch/README.md +++ b/plugin/output/elasticsearch/README.md @@ -116,7 +116,7 @@ Retention milliseconds for retry to DB. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index eeb488e13..4e8faaca0 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -160,7 +160,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } diff --git a/plugin/output/gelf/README.md b/plugin/output/gelf/README.md index 9ec4fcfc4..41ff37bc6 100755 --- a/plugin/output/gelf/README.md +++ b/plugin/output/gelf/README.md @@ -140,7 +140,7 @@ Retention milliseconds for retry to DB. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 6eb068702..5e21ddfae 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -165,7 +165,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // fields converted to extra fields GELF format diff --git a/plugin/output/kafka/README.md b/plugin/output/kafka/README.md index c9394e4bc..0dc0e3e05 100755 --- a/plugin/output/kafka/README.md +++ b/plugin/output/kafka/README.md @@ -79,7 +79,7 @@ Retention milliseconds for retry. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index 38bc76696..98a45a39e 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -116,7 +116,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * // > @3@4@5@6 diff --git a/plugin/output/postgres/README.md b/plugin/output/postgres/README.md index 40c2b49b8..9bfd08881 100755 --- a/plugin/output/postgres/README.md +++ b/plugin/output/postgres/README.md @@ -60,7 +60,7 @@ Retention milliseconds for retry to DB. **`db_request_timeout`** *`cfg.Duration`* *`default=3000ms`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries *`cfg.Duration`* *`default=3000ms`* Timeout for DB requests in milliseconds. diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index 80b260ef1..83bfcb787 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -174,7 +174,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // > @3@4@5@6 diff --git a/plugin/output/s3/README.md b/plugin/output/s3/README.md index d122b0500..5b914e248 100755 --- a/plugin/output/s3/README.md +++ b/plugin/output/s3/README.md @@ -166,7 +166,7 @@ Retention milliseconds for retry to upload. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index ce32d3303..78b381212 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -254,7 +254,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } diff --git a/plugin/output/splunk/README.md b/plugin/output/splunk/README.md index e6be14f53..5b456845a 100755 --- a/plugin/output/splunk/README.md +++ b/plugin/output/splunk/README.md @@ -67,7 +67,7 @@ Retention milliseconds for retry to DB. **`retention_exponentially_multiplier`** *`int`* *`default=2`* -Multiplier for exponentially increase retention beetween retries +Multiplier for exponential increase of retention between retries
diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index c99abdfb1..7f0dd6e5d 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -104,7 +104,7 @@ type Config struct { // > @3@4@5@6 // > - // > Multiplier for exponentially increase retention beetween retries + // > Multiplier for exponential increase of retention between retries RetentionExponentMultiplier int `json:"retention_exponentially_multiplier" default:"2"` // * } From c16e21d816446bacd56ad5f3d69d0951c7e7bcb0 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Mon, 15 Jan 2024 21:38:26 +0700 Subject: [PATCH 15/20] BatcherBackoff --- pipeline/backoff.go | 35 +++++++++++ pipeline/backoff_test.go | 47 +++++++++++++++ pipeline/batch.go | 21 ++++--- pipeline/batch_test.go | 7 ++- plugin/output/clickhouse/clickhouse.go | 54 ++++++++--------- plugin/output/elasticsearch/elasticsearch.go | 39 ++++++------ plugin/output/file/file.go | 5 +- plugin/output/gelf/gelf.go | 62 +++++++++----------- plugin/output/kafka/kafka.go | 43 ++++++-------- plugin/output/kafka/kafka_test.go | 5 +- plugin/output/postgres/postgres.go | 45 +++++++------- plugin/output/postgres/postgres_test.go | 26 +++----- plugin/output/s3/s3.go | 9 +-- plugin/output/splunk/splunk.go | 42 ++++++------- plugin/output/splunk/splunk_test.go | 6 +- 15 files changed, 247 insertions(+), 199 deletions(-) create mode 100644 pipeline/backoff.go create mode 100644 pipeline/backoff_test.go diff --git a/pipeline/backoff.go b/pipeline/backoff.go new file mode 100644 index 000000000..2f7e2881a --- /dev/null +++ b/pipeline/backoff.go @@ -0,0 +1,35 @@ +package pipeline + +import ( + "time" + + "github.com/cenkalti/backoff/v3" + "github.com/ozontech/file.d/cfg" +) + +type BatcherBackoff struct { + outFn BatcherOutFn + backoff backoff.BackOff + onRetryError func(err error) +} + +type BackoffOpts struct { + MinRetention time.Duration + Multiplier float64 + AttemptNum uint64 +} + +func NewBatcherBackoff(batcherOutFn BatcherOutFn, opts BackoffOpts, onRetryError func(err error)) *BatcherBackoff { + boff := cfg.GetBackoff(opts.MinRetention, opts.Multiplier, opts.AttemptNum) + return &BatcherBackoff{outFn: batcherOutFn, backoff: boff, onRetryError: onRetryError} +} + +func (b *BatcherBackoff) Out(data *WorkerData, batch *Batch) { + b.backoff.Reset() + err := backoff.Retry(func() error { + return b.outFn(data, batch) + }, b.backoff) + if err != nil { + b.onRetryError(err) + } +} diff --git a/pipeline/backoff_test.go b/pipeline/backoff_test.go new file mode 100644 index 000000000..2fceabdfb --- /dev/null +++ b/pipeline/backoff_test.go @@ -0,0 +1,47 @@ +package pipeline + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/assert" + "go.uber.org/atomic" +) + +func TestBackoff(t *testing.T) { + errorCount := &atomic.Int32{} + prevValue := errorCount.Load() + errorFn := func(err error) { + errorCount.Inc() + } + + batcherBackoff := NewBatcherBackoff( + func(workerData *WorkerData, batch *Batch) error { + return nil + }, + BackoffOpts{AttemptNum: 3}, + errorFn, + ) + + batcherBackoff.Out(nil, nil) + assert.Equal(t, prevValue, errorCount.Load(), "wrong error count") +} + +func TestBackoffWithError(t *testing.T) { + errorCount := &atomic.Int32{} + prevValue := errorCount.Load() + errorFn := func(err error) { + errorCount.Inc() + } + + batcherBackoff := NewBatcherBackoff( + func(workerData *WorkerData, batch *Batch) error { + return errors.New("some error") + }, + BackoffOpts{AttemptNum: 3}, + errorFn, + ) + + batcherBackoff.Out(nil, nil) + assert.Equal(t, prevValue+1, errorCount.Load(), "wrong error count") +} diff --git a/pipeline/batch.go b/pipeline/batch.go index ba4cf2428..1ac6278d8 100644 --- a/pipeline/batch.go +++ b/pipeline/batch.go @@ -5,8 +5,6 @@ import ( "sync" "time" - "github.com/cenkalti/backoff/v3" - "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/prometheus/client_golang/prometheus" @@ -135,7 +133,7 @@ type Batcher struct { } type ( - BatcherOutFn func(*WorkerData, *Batch, *backoff.BackOff) + BatcherOutFn func(*WorkerData, *Batch) error BatcherMaintenanceFn func(*WorkerData) BatcherOptions struct { @@ -153,6 +151,7 @@ type ( Retry int RetryRetention time.Duration RetryRetentionExponentMultiplier int + OnRetryError func(err error) } ) @@ -202,10 +201,15 @@ func (b *Batcher) work() { t := time.Now() data := WorkerData(nil) - workerBackoff := cfg.GetBackoff( - b.opts.RetryRetention, - float64(b.opts.RetryRetentionExponentMultiplier), - uint64(b.opts.Retry), + + workerBatcherBackoff := NewBatcherBackoff( + b.opts.OutFn, + BackoffOpts{ + MinRetention: b.opts.RetryRetention, + Multiplier: float64(b.opts.RetryRetentionExponentMultiplier), + AttemptNum: uint64(b.opts.Retry), + }, + b.opts.OnRetryError, ) for batch := range b.fullBatches { @@ -213,8 +217,7 @@ func (b *Batcher) work() { if batch.hasIterableEvents { now := time.Now() - workerBackoff.Reset() - b.opts.OutFn(&data, batch, &workerBackoff) + workerBatcherBackoff.Out(&data, batch) b.batchOutFnSeconds.Observe(time.Since(now).Seconds()) } diff --git a/pipeline/batch_test.go b/pipeline/batch_test.go index c00df3c75..659a34350 100644 --- a/pipeline/batch_test.go +++ b/pipeline/batch_test.go @@ -6,7 +6,6 @@ import ( "testing" "time" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/prometheus/client_golang/prometheus" @@ -37,12 +36,13 @@ func TestBatcher(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch, workerBackoff *backoff.BackOff) { + batcherOut := func(workerData *WorkerData, batch *Batch) error { if *workerData == nil { *workerData = batchCount } counter := (*workerData).(*atomic.Int32) counter.Inc() + return nil } seqIDs := make(map[SourceID]uint64) @@ -108,12 +108,13 @@ func TestBatcherMaxSize(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch, workerBackoff *backoff.BackOff) { + batcherOut := func(workerData *WorkerData, batch *Batch) error { if *workerData == nil { *workerData = batchCount } counter := (*workerData).(*atomic.Int32) counter.Inc() + return nil } seqIDs := make(map[SourceID]uint64) diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index e54b01f6e..c48d293a5 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -12,7 +12,6 @@ import ( "github.com/ClickHouse/ch-go" "github.com/ClickHouse/ch-go/chpool" "github.com/ClickHouse/ch-go/proto" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -21,6 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "go.uber.org/atomic" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -427,6 +427,18 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Log(level, "can't insert to the table", zap.Error(err), + zap.Int("retries", p.config.Retry), + zap.String("table", p.config.Table)) + }, }) p.batcher.Start(p.ctx) @@ -455,7 +467,7 @@ func (d data) reset() { } } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { // we don't check the error, schema already validated in the Start columns, _ := inferInsaneColInputs(p.config.Columns) @@ -495,36 +507,24 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor } }) - try := 0 - (*workerBackoff).Reset() - err := backoff.Retry(func() error { + var err error + for i := range p.instances { requestID := p.requestID.Inc() - clickhouse := p.getInstance(requestID, try) + clickhouse := p.getInstance(requestID, i) err := p.do(clickhouse, data.input) - if err != nil { - try++ - p.insertErrorsMetric.Inc() - p.logger.Error( - "an attempt to insert a batch failed", - zap.Int("try", try), - zap.Error(err), - ) + if err == nil { + return nil } - return err - }, *workerBackoff) - + } if err != nil { - var errLogFunc func(msg string, fields ...zap.Field) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal - } else { - errLogFunc = p.logger.Error - } - - errLogFunc("can't insert to the table", zap.Error(err), - zap.Int("retries", p.config.Retry), - zap.String("table", p.config.Table)) + p.insertErrorsMetric.Inc() + p.logger.Error( + "an attempt to insert a batch failed", + zap.Error(err), + ) } + + return err } func (p *Plugin) do(clickhouse Clickhouse, queryInput proto.Input) error { diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 4e8faaca0..74b89fd20 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -9,7 +9,6 @@ import ( "sync" "time" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/logger" @@ -20,6 +19,7 @@ import ( "github.com/valyala/fasthttp" insaneJSON "github.com/vitkovskii/insane-json" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -241,6 +241,18 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Log(level, "can't send to the elastic", zap.Error(err), + zap.Int("retries", p.config.Retry), + ) + }, }) ctx, cancel := context.WithCancel(context.Background()) @@ -263,7 +275,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.indexingErrorsMetric = ctl.RegisterCounter("output_elasticsearch_index_error", "Number of elasticsearch indexing errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -281,27 +293,12 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor data.outBuf = p.appendEvent(data.outBuf, event) }) - err := backoff.Retry(func() error { - err := p.send(data.outBuf) - if err != nil { - p.sendErrorMetric.Inc() - p.logger.Error("can't send to the elastic, will try other endpoint", zap.Error(err)) - } - return err - }, *workerBackoff) - + err := p.send(data.outBuf) if err != nil { - var errLogFunc func(msg string, fields ...zap.Field) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Sugar().Fatal - } else { - errLogFunc = p.logger.Sugar().Error - } - - errLogFunc("can't send to the elastic", zap.Error(err), - zap.Int("retries", p.config.Retry), - ) + p.sendErrorMetric.Inc() + p.logger.Error("can't send to the elastic, will try other endpoint", zap.Error(err)) } + return err } func (p *Plugin) send(body []byte) error { diff --git a/plugin/output/file/file.go b/plugin/output/file/file.go index 8761b1900..44f57d322 100644 --- a/plugin/output/file/file.go +++ b/plugin/output/file/file.go @@ -9,7 +9,6 @@ import ( "sync" "time" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/logger" @@ -182,7 +181,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -204,6 +203,8 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor data.outBuf = outBuf p.write(outBuf) + + return nil } func (p *Plugin) fileSealUpTicker(ctx context.Context) { diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 5e21ddfae..226ee17a8 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -5,7 +5,6 @@ import ( "strings" "time" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -13,6 +12,7 @@ import ( "github.com/prometheus/client_golang/prometheus" insaneJSON "github.com/vitkovskii/insane-json" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -229,6 +229,18 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't send to gelf", zap.Error(err), + zap.Int("retries", p.config.Retry), + ) + }, }) p.batcher.Start(context.TODO()) @@ -246,7 +258,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.sendErrorMetric = ctl.RegisterCounter("output_gelf_send_error", "Total GELF send errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -272,46 +284,30 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor data.outBuf = outBuf data.encodeBuf = encodeBuf - (*workerBackoff).Reset() - err := backoff.Retry(func() error { - if data.gelf == nil { - p.logger.Infof("connecting to gelf address=%s", p.config.Endpoint) - - gelf, err := newClient(p.config.Endpoint, p.config.ConnectionTimeout_, p.config.WriteTimeout_, false, nil) - if err != nil { - p.sendErrorMetric.Inc() - p.logger.Errorf("can't connect to gelf endpoint address=%s: %s", p.config.Endpoint, err.Error()) - time.Sleep(time.Second) - return err - } - data.gelf = gelf - } + if data.gelf == nil { + p.logger.Infof("connecting to gelf address=%s", p.config.Endpoint) - _, err := data.gelf.send(outBuf) + gelf, err := newClient(p.config.Endpoint, p.config.ConnectionTimeout_, p.config.WriteTimeout_, false, nil) if err != nil { p.sendErrorMetric.Inc() - p.logger.Errorf("can't send data to gelf address=%s, err: %s", p.config.Endpoint, err.Error()) - _ = data.gelf.close() - data.gelf = nil + p.logger.Errorf("can't connect to gelf endpoint address=%s: %s", p.config.Endpoint, err.Error()) time.Sleep(time.Second) return err } + data.gelf = gelf + } - return nil - }, *workerBackoff) - + _, err := data.gelf.send(outBuf) if err != nil { - var errLogFunc func(args ...interface{}) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal - } else { - errLogFunc = p.logger.Error - } - - errLogFunc("can't send to gelf", zap.Error(err), - zap.Int("retries", p.config.Retry), - ) + p.sendErrorMetric.Inc() + p.logger.Errorf("can't send data to gelf address=%s, err: %s", p.config.Endpoint, err.Error()) + _ = data.gelf.close() + data.gelf = nil + time.Sleep(time.Second) + return err } + + return nil } func (p *Plugin) maintenance(workerData *pipeline.WorkerData) { diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index 98a45a39e..d4de9b7bf 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -6,7 +6,6 @@ import ( "time" "github.com/Shopify/sarama" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -15,6 +14,7 @@ import ( "github.com/ozontech/file.d/xtls" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -193,6 +193,18 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't write batch", + zap.Int("retries", p.config.Retry), + ) + }, }) p.batcher.Start(context.TODO()) @@ -206,7 +218,7 @@ func (p *Plugin) registerMetrics(ctl *metric.Ctl) { p.sendErrorMetric = ctl.RegisterCounter("output_kafka_send_errors", "Total Kafka send errors") } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { *workerData = &data{ messages: make([]*sarama.ProducerMessage, p.config.BatchSize_), @@ -242,15 +254,8 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor i++ }) - data.outBuf = outBuf - - (*workerBackoff).Reset() - err := backoff.Retry(func() error { - err := p.producer.SendMessages(data.messages[:i]) - if err == nil { - return nil - } - + err := p.producer.SendMessages(data.messages[:i]) + if err != nil { errs := err.(sarama.ProducerErrors) for _, e := range errs { p.logger.Errorf("can't write batch: %s", e.Err.Error()) @@ -260,21 +265,9 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor "an attempt to insert a batch failed", zap.Error(err), ) - - return err - }, *workerBackoff) - - if err != nil { - var errLogFunc func(args ...interface{}) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal - } else { - errLogFunc = p.logger.Error - } - - errLogFunc("can't write batch", zap.Error(err), - zap.Int("retries", p.config.Retry)) } + + return err } func (p *Plugin) Stop() { diff --git a/plugin/output/kafka/kafka_test.go b/plugin/output/kafka/kafka_test.go index 748b19e10..552a494a3 100644 --- a/plugin/output/kafka/kafka_test.go +++ b/plugin/output/kafka/kafka_test.go @@ -6,10 +6,8 @@ package kafka import ( "fmt" "testing" - "time" "github.com/Shopify/sarama" - "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/pipeline" "github.com/stretchr/testify/require" insaneJSON "github.com/vitkovskii/insane-json" @@ -118,7 +116,6 @@ func FuzzKafka(f *testing.F) { producer: nil, batcher: nil, } - backoff := cfg.GetBackoff(1*time.Second, 1, 2) f.Fuzz(func(t *testing.T, topicField, topicVal, key, val string) { p.producer = &mockProducer{ @@ -128,6 +125,6 @@ func FuzzKafka(f *testing.F) { data := pipeline.NewPreparedBatch([]*pipeline.Event{ newEvent(t, topicField, topicVal, key, val), }) - p.out(&worker, data, &backoff) + p.out(&worker, data) }) } diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index 83bfcb787..4140b2e11 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -8,7 +8,6 @@ import ( "time" sq "github.com/Masterminds/squirrel" - "github.com/cenkalti/backoff/v3" "github.com/jackc/pgx/v4" "github.com/jackc/pgx/v4/pgxpool" "github.com/ozontech/file.d/cfg" @@ -18,6 +17,7 @@ import ( "github.com/prometheus/client_golang/prometheus" insaneJSON "github.com/vitkovskii/insane-json" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -288,6 +288,18 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't insert to the table", zap.Error(err), + zap.Int("retries", p.config.Retry), + zap.String("table", p.config.Table)) + }, }) ctx, cancel := context.WithCancel(context.Background()) @@ -307,7 +319,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch) error { // _ *pipeline.WorkerData - doesn't required in this plugin, we can't parse // events for uniques through bytes. builder := p.queryBuilder.GetInsertBuilder() @@ -352,7 +364,7 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch, workerBackof // no valid events passed. if !anyValidValue { - return + return nil } query, args, err := builder.ToSql() @@ -367,29 +379,14 @@ func (p *Plugin) out(_ *pipeline.WorkerData, batch *pipeline.Batch, workerBackof argsSliceInterface[i] = args[i-1] } - (*workerBackoff).Reset() - err = backoff.Retry(func() error { - err := p.try(query, argsSliceInterface) - if err != nil { - p.insertErrorsMetric.Inc() - p.logger.Errorf("can't exec query: %s", err.Error()) - return err - } - p.writtenEventMetric.Add(float64(len(uniqueEventsMap))) - return nil - }, *workerBackoff) - + err = p.try(query, argsSliceInterface) if err != nil { - var errLogFunc func(args ...interface{}) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal - p.pool.Close() - } else { - errLogFunc = p.logger.Error - } - - errLogFunc("failed insert into %s. query: %s, args: %v, err: %v", p.config.Table, query, args, err) + p.insertErrorsMetric.Inc() + p.logger.Errorf("can't exec query: %s", err.Error()) + return err } + p.writtenEventMetric.Add(float64(len(uniqueEventsMap))) + return nil } func (p *Plugin) try(query string, argsSliceInterface []any) error { diff --git a/plugin/output/postgres/postgres_test.go b/plugin/output/postgres/postgres_test.go index 859a2e701..53d905ddd 100644 --- a/plugin/output/postgres/postgres_test.go +++ b/plugin/output/postgres/postgres_test.go @@ -10,7 +10,6 @@ import ( "github.com/golang/mock/gomock" "github.com/jackc/pgconn" "github.com/jackc/pgproto3/v2" - "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/logger" "github.com/ozontech/file.d/metric" "github.com/ozontech/file.d/pipeline" @@ -94,8 +93,7 @@ func TestPrivateOut(t *testing.T) { p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } func TestPrivateOutWithRetry(t *testing.T) { @@ -149,12 +147,7 @@ func TestPrivateOutWithRetry(t *testing.T) { gomock.AssignableToTypeOf(ctxMock), "INSERT INTO table1 (str_uni_1,int_1,timestamp_1) VALUES ($1,$2,$3) ON CONFLICT(str_uni_1) DO UPDATE SET int_1=EXCLUDED.int_1,timestamp_1=EXCLUDED.timestamp_1", []any{preferSimpleProtocol, strUniValue, intValue, time.Unix(int64(timestampValue), 0).Format(time.RFC3339)}, - ).Return(&rowsForTest{}, errors.New("someError")).Times(2) - mockpool.EXPECT().Query( - gomock.AssignableToTypeOf(ctxMock), - "INSERT INTO table1 (str_uni_1,int_1,timestamp_1) VALUES ($1,$2,$3) ON CONFLICT(str_uni_1) DO UPDATE SET int_1=EXCLUDED.int_1,timestamp_1=EXCLUDED.timestamp_1", - []any{preferSimpleProtocol, strUniValue, intValue, time.Unix(int64(timestampValue), 0).Format(time.RFC3339)}, - ).Return(&rowsForTest{}, nil).Times(1) + ).Return(&rowsForTest{}, errors.New("someError")).Times(1) builder, err := NewQueryBuilder(columns, table) require.NoError(t, err) @@ -170,8 +163,7 @@ func TestPrivateOutWithRetry(t *testing.T) { p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } func TestPrivateOutNoGoodEvents(t *testing.T) { @@ -224,8 +216,7 @@ func TestPrivateOutNoGoodEvents(t *testing.T) { p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } func TestPrivateOutDeduplicatedEvents(t *testing.T) { @@ -309,8 +300,7 @@ func TestPrivateOutDeduplicatedEvents(t *testing.T) { {Root: rootDuplication}, {Root: rootDuplicationMore}, }) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } func TestPrivateOutWrongTypeInField(t *testing.T) { @@ -372,8 +362,7 @@ func TestPrivateOutWrongTypeInField(t *testing.T) { p.registerMetrics(metric.NewCtl("test", prometheus.NewRegistry())) batch := pipeline.NewPreparedBatch([]*pipeline.Event{{Root: root}}) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *testing.T) { @@ -484,8 +473,7 @@ func TestPrivateOutFewUniqueEventsYetWithDeduplicationEventsAnpooladEvents(t *te {Root: secondUniqueRoot}, {Root: badRoot}, }) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - p.out(nil, batch, &backoff) + p.out(nil, batch) } // TODO replace with gomock diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index 78b381212..88ed4b79d 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -22,6 +22,7 @@ import ( "github.com/ozontech/file.d/plugin/output/file" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -552,14 +553,14 @@ func (p *Plugin) uploadWork(workerBackoff backoff.BackOff) { }, workerBackoff) if err != nil { - var errLogFunc func(args ...interface{}) + var level zapcore.Level if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal + level = zapcore.FatalLevel } else { - errLogFunc = p.logger.Error + level = zapcore.ErrorLevel } - errLogFunc("could not upload s3 object", zap.Error(err), + p.logger.Desugar().Log(level, "could not upload s3 object", zap.Error(err), zap.Int("retries", p.config.Retry), ) } diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index 7f0dd6e5d..cef2286b9 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -10,7 +10,6 @@ import ( "net/http" "time" - "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" "github.com/ozontech/file.d/metric" @@ -18,6 +17,7 @@ import ( "github.com/prometheus/client_golang/prometheus" insaneJSON "github.com/vitkovskii/insane-json" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) /*{ introduction @@ -145,6 +145,17 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP Retry: p.config.Retry, RetryRetention: p.config.Retention_, RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, + OnRetryError: func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't send data to splunk", zap.Error(err), + zap.Int("retries", p.config.Retry)) + }, }) p.batcher.Start(context.TODO()) @@ -162,7 +173,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, workerBackoff *backoff.BackOff) { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -188,30 +199,15 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch, wor p.logger.Debugf("trying to send: %s", outBuf) - (*workerBackoff).Reset() - err := backoff.Retry(func() error { - err := p.send(outBuf) - if err != nil { - p.sendErrorMetric.Inc() - p.logger.Errorf("can't send data to splunk address=%s: %s", p.config.Endpoint, err.Error()) - } - return err - }, *workerBackoff) - + err := p.send(outBuf) if err != nil { - var errLogFunc func(args ...interface{}) - if p.config.FatalOnFailedInsert { - errLogFunc = p.logger.Fatal - } else { - errLogFunc = p.logger.Error - } - - errLogFunc("can't send data to splunk", zap.Error(err), - zap.Int("retries", p.config.Retry), - ) + p.sendErrorMetric.Inc() + p.logger.Errorf("can't send data to splunk address=%s: %s", p.config.Endpoint, err.Error()) + } else { + p.logger.Debugf("successfully sent: %s", outBuf) } - p.logger.Debugf("successfully sent: %s", outBuf) + return err } func (p *Plugin) maintenance(_ *pipeline.WorkerData) {} diff --git a/plugin/output/splunk/splunk_test.go b/plugin/output/splunk/splunk_test.go index 7258b0ed5..2661ecf48 100644 --- a/plugin/output/splunk/splunk_test.go +++ b/plugin/output/splunk/splunk_test.go @@ -5,9 +5,7 @@ import ( "net/http" "net/http/httptest" "testing" - "time" - "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/pipeline" "github.com/stretchr/testify/assert" insaneJSON "github.com/vitkovskii/insane-json" @@ -58,10 +56,8 @@ func TestSplunk(t *testing.T) { {Root: input}, }) - backoff := cfg.GetBackoff(1*time.Second, 1, 2) - data := pipeline.WorkerData(nil) - plugin.out(&data, batch, &backoff) + plugin.out(&data, batch) assert.Equal(t, testCase.expected+testCase.expected, string(response)) }) From 16aa6806c65367c927fb7b7a9b92649bf6476640 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Tue, 23 Jan 2024 15:18:58 +0700 Subject: [PATCH 16/20] revert batcher --- pipeline/batch.go | 41 +++++++++++++---------------------------- pipeline/batch_test.go | 6 ++---- 2 files changed, 15 insertions(+), 32 deletions(-) diff --git a/pipeline/batch.go b/pipeline/batch.go index 1ac6278d8..4fb0380fa 100644 --- a/pipeline/batch.go +++ b/pipeline/batch.go @@ -133,25 +133,21 @@ type Batcher struct { } type ( - BatcherOutFn func(*WorkerData, *Batch) error + BatcherOutFn func(*WorkerData, *Batch) BatcherMaintenanceFn func(*WorkerData) BatcherOptions struct { - PipelineName string - OutputType string - OutFn BatcherOutFn - MaintenanceFn BatcherMaintenanceFn - Controller OutputPluginController - Workers int - BatchSizeCount int - BatchSizeBytes int - FlushTimeout time.Duration - MaintenanceInterval time.Duration - MetricCtl *metric.Ctl - Retry int - RetryRetention time.Duration - RetryRetentionExponentMultiplier int - OnRetryError func(err error) + PipelineName string + OutputType string + OutFn BatcherOutFn + MaintenanceFn BatcherMaintenanceFn + Controller OutputPluginController + Workers int + BatchSizeCount int + BatchSizeBytes int + FlushTimeout time.Duration + MaintenanceInterval time.Duration + MetricCtl *metric.Ctl } ) @@ -201,23 +197,12 @@ func (b *Batcher) work() { t := time.Now() data := WorkerData(nil) - - workerBatcherBackoff := NewBatcherBackoff( - b.opts.OutFn, - BackoffOpts{ - MinRetention: b.opts.RetryRetention, - Multiplier: float64(b.opts.RetryRetentionExponentMultiplier), - AttemptNum: uint64(b.opts.Retry), - }, - b.opts.OnRetryError, - ) - for batch := range b.fullBatches { b.workersInProgress.Inc() if batch.hasIterableEvents { now := time.Now() - workerBatcherBackoff.Out(&data, batch) + b.opts.OutFn(&data, batch) b.batchOutFnSeconds.Observe(time.Since(now).Seconds()) } diff --git a/pipeline/batch_test.go b/pipeline/batch_test.go index 659a34350..8ae5fae57 100644 --- a/pipeline/batch_test.go +++ b/pipeline/batch_test.go @@ -36,13 +36,12 @@ func TestBatcher(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch) error { + batcherOut := func(workerData *WorkerData, batch *Batch) { if *workerData == nil { *workerData = batchCount } counter := (*workerData).(*atomic.Int32) counter.Inc() - return nil } seqIDs := make(map[SourceID]uint64) @@ -108,13 +107,12 @@ func TestBatcherMaxSize(t *testing.T) { wg.Add(eventCount) batchCount := &atomic.Int32{} - batcherOut := func(workerData *WorkerData, batch *Batch) error { + batcherOut := func(workerData *WorkerData, batch *Batch) { if *workerData == nil { *workerData = batchCount } counter := (*workerData).(*atomic.Int32) counter.Inc() - return nil } seqIDs := make(map[SourceID]uint64) From a87398172d620ab5cee7009e353e55765cebcb6c Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Tue, 23 Jan 2024 17:51:28 +0700 Subject: [PATCH 17/20] RetriableBatcher --- pipeline/backoff.go | 46 +++++++++++-- pipeline/backoff_test.go | 23 +++++-- plugin/output/clickhouse/clickhouse.go | 62 ++++++++++-------- plugin/output/elasticsearch/elasticsearch.go | 67 +++++++++++-------- plugin/output/file/file.go | 4 +- plugin/output/gelf/gelf.go | 68 +++++++++++--------- plugin/output/kafka/kafka.go | 63 ++++++++++-------- plugin/output/postgres/postgres.go | 62 ++++++++++-------- plugin/output/splunk/splunk.go | 64 ++++++++++-------- 9 files changed, 284 insertions(+), 175 deletions(-) diff --git a/pipeline/backoff.go b/pipeline/backoff.go index 2f7e2881a..4def59046 100644 --- a/pipeline/backoff.go +++ b/pipeline/backoff.go @@ -1,35 +1,69 @@ package pipeline import ( + "context" "time" "github.com/cenkalti/backoff/v3" "github.com/ozontech/file.d/cfg" ) -type BatcherBackoff struct { - outFn BatcherOutFn +type RetriableBatcher struct { + outFn RetriableBatcherOutFn backoff backoff.BackOff + batcher *Batcher onRetryError func(err error) } +type RetriableBatcherOutFn func(*WorkerData, *Batch) error + type BackoffOpts struct { MinRetention time.Duration Multiplier float64 AttemptNum uint64 } -func NewBatcherBackoff(batcherOutFn BatcherOutFn, opts BackoffOpts, onRetryError func(err error)) *BatcherBackoff { - boff := cfg.GetBackoff(opts.MinRetention, opts.Multiplier, opts.AttemptNum) - return &BatcherBackoff{outFn: batcherOutFn, backoff: boff, onRetryError: onRetryError} +func NewRetriableBatcher(batcherOpts *BatcherOptions, batcherOutFn RetriableBatcherOutFn, opts BackoffOpts, onError func(err error)) *RetriableBatcher { + boff := cfg.GetBackoff( + opts.MinRetention, + opts.Multiplier, + opts.AttemptNum, + ) + + batcherBackoff := &RetriableBatcher{ + outFn: batcherOutFn, + backoff: boff, + onRetryError: onError, + } + batcherBackoff.setBatcher(batcherOpts) + return batcherBackoff } -func (b *BatcherBackoff) Out(data *WorkerData, batch *Batch) { +func (b *RetriableBatcher) setBatcher(batcherOpts *BatcherOptions) { + batcherOpts.OutFn = b.Out + b.batcher = NewBatcher(*batcherOpts) +} + +func (b *RetriableBatcher) Out(data *WorkerData, batch *Batch) { b.backoff.Reset() + err := backoff.Retry(func() error { return b.outFn(data, batch) }, b.backoff) + if err != nil { b.onRetryError(err) } } + +func (b *RetriableBatcher) Start(ctx context.Context) { + b.batcher.Start(ctx) +} + +func (b *RetriableBatcher) Stop() { + b.batcher.Stop() +} + +func (b *RetriableBatcher) Add(event *Event) { + b.batcher.Add(event) +} diff --git a/pipeline/backoff_test.go b/pipeline/backoff_test.go index 2fceabdfb..8f700eb6d 100644 --- a/pipeline/backoff_test.go +++ b/pipeline/backoff_test.go @@ -4,19 +4,29 @@ import ( "errors" "testing" + "github.com/ozontech/file.d/metric" + "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/assert" "go.uber.org/atomic" ) func TestBackoff(t *testing.T) { errorCount := &atomic.Int32{} - prevValue := errorCount.Load() + errorCountBefore := errorCount.Load() + + eventCount := &atomic.Int32{} + eventCountBefore := eventCount.Load() + errorFn := func(err error) { errorCount.Inc() } - batcherBackoff := NewBatcherBackoff( + batcherBackoff := NewRetriableBatcher( + &BatcherOptions{ + MetricCtl: metric.New("", prometheus.NewRegistry()), + }, func(workerData *WorkerData, batch *Batch) error { + eventCount.Inc() return nil }, BackoffOpts{AttemptNum: 3}, @@ -24,7 +34,9 @@ func TestBackoff(t *testing.T) { ) batcherBackoff.Out(nil, nil) - assert.Equal(t, prevValue, errorCount.Load(), "wrong error count") + + assert.Equal(t, errorCountBefore, errorCount.Load(), "wrong error count") + assert.Equal(t, eventCountBefore+1, eventCount.Load(), "wrong event count") } func TestBackoffWithError(t *testing.T) { @@ -34,7 +46,10 @@ func TestBackoffWithError(t *testing.T) { errorCount.Inc() } - batcherBackoff := NewBatcherBackoff( + batcherBackoff := NewRetriableBatcher( + &BatcherOptions{ + MetricCtl: metric.New("", prometheus.NewRegistry()), + }, func(workerData *WorkerData, batch *Batch) error { return errors.New("some error") }, diff --git a/plugin/output/clickhouse/clickhouse.go b/plugin/output/clickhouse/clickhouse.go index c48d293a5..25ef5e8ac 100644 --- a/plugin/output/clickhouse/clickhouse.go +++ b/plugin/output/clickhouse/clickhouse.go @@ -44,7 +44,7 @@ type Plugin struct { logger *zap.Logger config *Config - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher ctx context.Context cancelFunc context.CancelFunc @@ -414,32 +414,42 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP } } - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: params.Controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + Controller: params.Controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + } - p.logger.Log(level, "can't insert to the table", zap.Error(err), - zap.Int("retries", p.config.Retry), - zap.String("table", p.config.Table)) - }, - }) + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Log(level, "can't insert to the table", zap.Error(err), + zap.Int("retries", p.config.Retry), + zap.String("table", p.config.Table)) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) p.batcher.Start(p.ctx) } diff --git a/plugin/output/elasticsearch/elasticsearch.go b/plugin/output/elasticsearch/elasticsearch.go index 74b89fd20..4553cf59b 100644 --- a/plugin/output/elasticsearch/elasticsearch.go +++ b/plugin/output/elasticsearch/elasticsearch.go @@ -46,7 +46,7 @@ type Plugin struct { avgEventSize int time string headerPrefix string - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher controller pipeline.OutputPluginController mu *sync.Mutex @@ -226,34 +226,45 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.maintenance(nil) p.logger.Info("starting batcher", zap.Duration("timeout", p.config.BatchFlushTimeout_)) - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MaintenanceInterval: time.Minute, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } - p.logger.Log(level, "can't send to the elastic", zap.Error(err), - zap.Int("retries", p.config.Retry), - ) - }, - }) + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MaintenanceInterval: time.Minute, + MetricCtl: params.MetricCtl, + } + + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Log(level, "can't send to the elastic", zap.Error(err), + zap.Int("retries", p.config.Retry), + ) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) ctx, cancel := context.WithCancel(context.Background()) p.cancel = cancel diff --git a/plugin/output/file/file.go b/plugin/output/file/file.go index 44f57d322..8311e08da 100644 --- a/plugin/output/file/file.go +++ b/plugin/output/file/file.go @@ -181,7 +181,7 @@ func (p *Plugin) Out(event *pipeline.Event) { p.batcher.Add(event) } -func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) error { +func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) { if *workerData == nil { *workerData = &data{ outBuf: make([]byte, 0, p.config.BatchSize_*p.avgEventSize), @@ -203,8 +203,6 @@ func (p *Plugin) out(workerData *pipeline.WorkerData, batch *pipeline.Batch) err data.outBuf = outBuf p.write(outBuf) - - return nil } func (p *Plugin) fileSealUpTicker(ctx context.Context) { diff --git a/plugin/output/gelf/gelf.go b/plugin/output/gelf/gelf.go index 226ee17a8..95757151a 100644 --- a/plugin/output/gelf/gelf.go +++ b/plugin/output/gelf/gelf.go @@ -42,7 +42,7 @@ type Plugin struct { config *Config logger *zap.SugaredLogger avgEventSize int - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher controller pipeline.OutputPluginController // plugin metrics @@ -214,34 +214,44 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.config.timestampFieldFormat = format p.config.levelField = pipeline.ByteToStringUnsafe(p.formatExtraField(nil, p.config.LevelField)) - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MaintenanceInterval: p.config.ReconnectInterval_, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } - - p.logger.Desugar().Log(level, "can't send to gelf", zap.Error(err), - zap.Int("retries", p.config.Retry), - ) - }, - }) + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MaintenanceInterval: p.config.ReconnectInterval_, + MetricCtl: params.MetricCtl, + } + + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't send to gelf", zap.Error(err), + zap.Int("retries", p.config.Retry), + ) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) p.batcher.Start(context.TODO()) } diff --git a/plugin/output/kafka/kafka.go b/plugin/output/kafka/kafka.go index d4de9b7bf..ef880beb8 100644 --- a/plugin/output/kafka/kafka.go +++ b/plugin/output/kafka/kafka.go @@ -37,7 +37,7 @@ type Plugin struct { controller pipeline.OutputPluginController producer sarama.SyncProducer - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher // plugin metrics sendErrorMetric prometheus.Counter @@ -180,32 +180,43 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.logger.Infof("workers count=%d, batch size=%d", p.config.WorkersCount_, p.config.BatchSize_) p.producer = NewProducer(p.config, p.logger) - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } - p.logger.Desugar().Log(level, "can't write batch", - zap.Int("retries", p.config.Retry), - ) - }, - }) + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + } + + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't write batch", + zap.Int("retries", p.config.Retry), + ) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) p.batcher.Start(context.TODO()) } diff --git a/plugin/output/postgres/postgres.go b/plugin/output/postgres/postgres.go index 4140b2e11..531396b75 100644 --- a/plugin/output/postgres/postgres.go +++ b/plugin/output/postgres/postgres.go @@ -98,7 +98,7 @@ type Plugin struct { controller pipeline.OutputPluginController logger *zap.SugaredLogger config *Config - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher ctx context.Context cancelFunc context.CancelFunc @@ -275,32 +275,42 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP } p.pool = pool - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + } - p.logger.Desugar().Log(level, "can't insert to the table", zap.Error(err), - zap.Int("retries", p.config.Retry), - zap.String("table", p.config.Table)) - }, - }) + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't insert to the table", zap.Error(err), + zap.Int("retries", p.config.Retry), + zap.String("table", p.config.Table)) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) ctx, cancel := context.WithCancel(context.Background()) p.ctx = ctx diff --git a/plugin/output/splunk/splunk.go b/plugin/output/splunk/splunk.go index cef2286b9..99cc07e8c 100644 --- a/plugin/output/splunk/splunk.go +++ b/plugin/output/splunk/splunk.go @@ -33,7 +33,7 @@ type Plugin struct { client http.Client logger *zap.SugaredLogger avgEventSize int - batcher *pipeline.Batcher + batcher *pipeline.RetriableBatcher controller pipeline.OutputPluginController // plugin metrics @@ -131,32 +131,42 @@ func (p *Plugin) Start(config pipeline.AnyConfig, params *pipeline.OutputPluginP p.registerMetrics(params.MetricCtl) p.client = p.newClient(p.config.RequestTimeout_) - p.batcher = pipeline.NewBatcher(pipeline.BatcherOptions{ - PipelineName: params.PipelineName, - OutputType: outPluginType, - OutFn: p.out, - MaintenanceFn: p.maintenance, - Controller: p.controller, - Workers: p.config.WorkersCount_, - BatchSizeCount: p.config.BatchSize_, - BatchSizeBytes: p.config.BatchSizeBytes_, - FlushTimeout: p.config.BatchFlushTimeout_, - MetricCtl: params.MetricCtl, - Retry: p.config.Retry, - RetryRetention: p.config.Retention_, - RetryRetentionExponentMultiplier: p.config.RetentionExponentMultiplier, - OnRetryError: func(err error) { - var level zapcore.Level - if p.config.FatalOnFailedInsert { - level = zapcore.FatalLevel - } else { - level = zapcore.ErrorLevel - } - - p.logger.Desugar().Log(level, "can't send data to splunk", zap.Error(err), - zap.Int("retries", p.config.Retry)) - }, - }) + batcherOpts := pipeline.BatcherOptions{ + PipelineName: params.PipelineName, + OutputType: outPluginType, + MaintenanceFn: p.maintenance, + Controller: p.controller, + Workers: p.config.WorkersCount_, + BatchSizeCount: p.config.BatchSize_, + BatchSizeBytes: p.config.BatchSizeBytes_, + FlushTimeout: p.config.BatchFlushTimeout_, + MetricCtl: params.MetricCtl, + } + + backoffOpts := pipeline.BackoffOpts{ + MinRetention: p.config.Retention_, + Multiplier: float64(p.config.RetentionExponentMultiplier), + AttemptNum: uint64(p.config.Retry), + } + + onError := func(err error) { + var level zapcore.Level + if p.config.FatalOnFailedInsert { + level = zapcore.FatalLevel + } else { + level = zapcore.ErrorLevel + } + + p.logger.Desugar().Log(level, "can't send data to splunk", zap.Error(err), + zap.Int("retries", p.config.Retry)) + } + + p.batcher = pipeline.NewRetriableBatcher( + &batcherOpts, + p.out, + backoffOpts, + onError, + ) p.batcher.Start(context.TODO()) } From 7b64021beacb547bc5a31e94828f33e09035ac8f Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Wed, 24 Jan 2024 19:38:26 +0700 Subject: [PATCH 18/20] move Backoff from cfg package --- cfg/backoff.go | 15 --------------- pipeline/backoff.go | 11 +++++++++-- plugin/output/s3/s3.go | 2 +- 3 files changed, 10 insertions(+), 18 deletions(-) delete mode 100644 cfg/backoff.go diff --git a/cfg/backoff.go b/cfg/backoff.go deleted file mode 100644 index c96be65e4..000000000 --- a/cfg/backoff.go +++ /dev/null @@ -1,15 +0,0 @@ -package cfg - -import ( - "time" - - "github.com/cenkalti/backoff/v3" -) - -func GetBackoff(minRetention time.Duration, multiplier float64, attemptNum uint64) backoff.BackOff { - expBackoff := backoff.NewExponentialBackOff() - expBackoff.InitialInterval = minRetention - expBackoff.Multiplier = multiplier - expBackoff.RandomizationFactor = 0.5 - return backoff.WithMaxRetries(expBackoff, attemptNum) -} diff --git a/pipeline/backoff.go b/pipeline/backoff.go index 4def59046..5827136dd 100644 --- a/pipeline/backoff.go +++ b/pipeline/backoff.go @@ -5,7 +5,6 @@ import ( "time" "github.com/cenkalti/backoff/v3" - "github.com/ozontech/file.d/cfg" ) type RetriableBatcher struct { @@ -24,7 +23,7 @@ type BackoffOpts struct { } func NewRetriableBatcher(batcherOpts *BatcherOptions, batcherOutFn RetriableBatcherOutFn, opts BackoffOpts, onError func(err error)) *RetriableBatcher { - boff := cfg.GetBackoff( + boff := GetBackoff( opts.MinRetention, opts.Multiplier, opts.AttemptNum, @@ -67,3 +66,11 @@ func (b *RetriableBatcher) Stop() { func (b *RetriableBatcher) Add(event *Event) { b.batcher.Add(event) } + +func GetBackoff(minRetention time.Duration, multiplier float64, attemptNum uint64) backoff.BackOff { + expBackoff := backoff.NewExponentialBackOff() + expBackoff.InitialInterval = minRetention + expBackoff.Multiplier = multiplier + expBackoff.RandomizationFactor = 0.5 + return backoff.WithMaxRetries(expBackoff, attemptNum) +} diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index 88ed4b79d..6786e4d7a 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -336,7 +336,7 @@ func (p *Plugin) StartWithMinio(config pipeline.AnyConfig, params *pipeline.Outp p.compressCh = make(chan fileDTO, p.config.FileConfig.WorkersCount_) for i := 0; i < p.config.FileConfig.WorkersCount_; i++ { - go p.uploadWork(cfg.GetBackoff( + go p.uploadWork(pipeline.GetBackoff( p.config.Retention_, float64(p.config.RetentionExponentMultiplier), uint64(p.config.Retry), From 099cc6559e817b26e76eaae280a56d794bb954d0 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Tue, 30 Jan 2024 13:06:19 +0700 Subject: [PATCH 19/20] use backoff/v4 --- go.mod | 1 + go.sum | 2 ++ pipeline/backoff.go | 2 +- plugin/output/s3/s3.go | 2 +- 4 files changed, 5 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 68299446f..1fe3bbcf9 100644 --- a/go.mod +++ b/go.mod @@ -15,6 +15,7 @@ require ( github.com/bitly/go-simplejson v0.5.1 github.com/cespare/xxhash/v2 v2.2.0 github.com/cenkalti/backoff/v3 v3.0.0 + github.com/cenkalti/backoff/v4 v4.2.1 github.com/euank/go-kmsg-parser v2.0.0+incompatible github.com/go-redis/redis v6.15.9+incompatible github.com/golang/mock v1.6.0 diff --git a/go.sum b/go.sum index aeba12cfb..5312332e4 100644 --- a/go.sum +++ b/go.sum @@ -33,6 +33,8 @@ github.com/bitly/go-simplejson v0.5.1 h1:xgwPbetQScXt1gh9BmoJ6j9JMr3TElvuIyjR8pg github.com/bitly/go-simplejson v0.5.1/go.mod h1:YOPVLzCfwK14b4Sff3oP1AmGhI9T9Vsg84etUnlyp+Q= github.com/cenkalti/backoff/v3 v3.0.0 h1:ske+9nBpD9qZsTBoF41nW5L+AIuFBKMeze18XQ3eG1c= github.com/cenkalti/backoff/v3 v3.0.0/go.mod h1:cIeZDE3IrqwwJl6VUwCN6trj1oXrTS4rc0ij+ULvLYs= +github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= +github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= diff --git a/pipeline/backoff.go b/pipeline/backoff.go index 5827136dd..af3401366 100644 --- a/pipeline/backoff.go +++ b/pipeline/backoff.go @@ -4,7 +4,7 @@ import ( "context" "time" - "github.com/cenkalti/backoff/v3" + "github.com/cenkalti/backoff/v4" ) type RetriableBatcher struct { diff --git a/plugin/output/s3/s3.go b/plugin/output/s3/s3.go index 6786e4d7a..84148db68 100644 --- a/plugin/output/s3/s3.go +++ b/plugin/output/s3/s3.go @@ -13,7 +13,7 @@ import ( "sync" "time" - "github.com/cenkalti/backoff/v3" + "github.com/cenkalti/backoff/v4" "github.com/minio/minio-go" "github.com/ozontech/file.d/cfg" "github.com/ozontech/file.d/fd" From ad1526d91b5ff5501711f1c8a55d31f4735568f2 Mon Sep 17 00:00:00 2001 From: Dmitry Romanov Date: Wed, 31 Jan 2024 18:07:30 +0700 Subject: [PATCH 20/20] fix TestBackoff --- go.mod | 4 +--- pipeline/backoff_test.go | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 1fe3bbcf9..590f36e84 100644 --- a/go.mod +++ b/go.mod @@ -13,9 +13,8 @@ require ( github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 github.com/alicebob/miniredis/v2 v2.30.5 github.com/bitly/go-simplejson v0.5.1 - github.com/cespare/xxhash/v2 v2.2.0 - github.com/cenkalti/backoff/v3 v3.0.0 github.com/cenkalti/backoff/v4 v4.2.1 + github.com/cespare/xxhash/v2 v2.2.0 github.com/euank/go-kmsg-parser v2.0.0+incompatible github.com/go-redis/redis v6.15.9+incompatible github.com/golang/mock v1.6.0 @@ -51,7 +50,6 @@ require ( github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v3 v3.0.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cilium/ebpf v0.9.1 // indirect github.com/containerd/cgroups/v3 v3.0.1 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect diff --git a/pipeline/backoff_test.go b/pipeline/backoff_test.go index 8f700eb6d..1d73a2bf1 100644 --- a/pipeline/backoff_test.go +++ b/pipeline/backoff_test.go @@ -23,7 +23,7 @@ func TestBackoff(t *testing.T) { batcherBackoff := NewRetriableBatcher( &BatcherOptions{ - MetricCtl: metric.New("", prometheus.NewRegistry()), + MetricCtl: metric.NewCtl("", prometheus.NewRegistry()), }, func(workerData *WorkerData, batch *Batch) error { eventCount.Inc() @@ -48,7 +48,7 @@ func TestBackoffWithError(t *testing.T) { batcherBackoff := NewRetriableBatcher( &BatcherOptions{ - MetricCtl: metric.New("", prometheus.NewRegistry()), + MetricCtl: metric.NewCtl("", prometheus.NewRegistry()), }, func(workerData *WorkerData, batch *Batch) error { return errors.New("some error")