From bef231168a49ffbe502de8cfcb61093c361eca12 Mon Sep 17 00:00:00 2001 From: Travis Bischel Date: Sat, 20 Mar 2021 13:24:10 -0600 Subject: [PATCH] enforce KIP-98 idempotency config rules: acks=all, retries > 0 We do not enforce max inflight requests per connection because our client internally bounds that to 1 unless we notice we are on Kafka 1.0.0, where the limit was raised to 5. --- pkg/kgo/config.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pkg/kgo/config.go b/pkg/kgo/config.go index a545beb5..aedc38a1 100644 --- a/pkg/kgo/config.go +++ b/pkg/kgo/config.go @@ -121,6 +121,14 @@ func (cfg *cfg) validate() error { if cfg.disableIdempotency && cfg.txnID != nil { return errors.New("cannot both disable idempotent writes and use transactional IDs") } + if !cfg.disableIdempotency { + if cfg.acks.val != -1 { + return errors.New("idempotency requires acks=all") + } + if cfg.retries == 0 { + return errors.New("idempotency requires RequestRetries to be greater than 0") + } + } for _, limit := range []struct { name string