Details
Description
We increase our cluster capacity from 50 brokers to 80 brokers.
We do a broker partition reassign while producers is sending message. After finished we found a small number of producer in a infinite retry on OutOfOrderSequenceException. It's recover when we restart problem producer(ask for a new PID).
We found problem partition error log in broker server.log like:
ERROR [ReplicaManager broker=79] Error processing append operation on partition test-76 (kafka.server.ReplicaManager)
org.apache.kafka.common.errors.OutOfOrderSequenceException: Out of order sequence number for producerId 140981: 834530 (incoming seq. number), 834543 (current end sequence number)
Strange things is the incoming seq. number is smaller than borker current end sequence number. Before this exception problem partition has do a leader election.
[17:08:20,706] INFO [Partition test-76 broker=79] test-76 starts at Leader Epoch 2 from offset 217709710. Previous Leader Epoch was: 1 (kafka.cluster.Partition)
[17:08:20,715] INFO [Partition test-76 broker=79] test-76 starts at Leader Epoch 6 from offset 217709710. Previous Leader Epoch was: 2 (kafka.cluster.Partition)
And in producer side, it has NETWORK_EXCEPTION before into OutOfOrderSequenceException. So we think maybe some message send success to broker, but not response to producer. After partition leader change producer retry those old message always reject by broker because of the OutOfOrderSequenceException.
Our primary producer config:
enable.idempotence = true
retries = Integer.MAX_VALUE
acks = all
max.in.flight.requests.per.connection = 5
compression.type = lz4
metadata.max.age.ms = 300000
Topic config:
min.insync.replicas = 2
4 replicas each partition