[ https://issues.apache.org/jira/browse/KAFKA-5678?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16112124#comment-16112124 ]
cuiyang edited comment on KAFKA-5678 at 8/3/17 3:35 AM: -------------------------------------------------------- [~becket_qin] Hi Jiang, thank you for your patience, I have two confusions to your explanation: 1 I think producer doesn't try on request timeout: RecordAccumulator::abortExpiredBatches() -> batch.expirationDone(); -> this.done() -> thunk.callback.onCompletion() -> this.userCallback.onCompletion() //will remove the batch from queue and invoke the user callback without any retries. So this is why we so care about the request timeout rather than other errors, because retries doesn't work for it even we set retries to non-zero. 2 Even if we only make leader migration without a broker going down(assuming that we migrate all the leaders from Broker A to other Broker B, C,D), we still need face the scenario which is described by [~Json Tu]: bg."if there are some partition leader did not change in this request but not have enough replica, then it will not satisfy such code as below." if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) forceComplete() was (Author: cuiyang): [~becket_qin] Hi Jiang, thank you for your patience, I have two confusions to your explanation: 1 I think producer doesn't try on request timeout: RecordAccumulator::abortExpiredBatches() -> batch.expirationDone(); -> this.done() -> thunk.callback.onCompletion() -> this.userCallback.onCompletion() //will remove the batch from queue and invoke the user callback without any retries. So this is why we so care about the request timeout rather than other errors, because retries doesn't work for it even we set retries to non-zero. 2 Even if we only make leader migration without a broker going down(assuming that we migrate all the leaders from Broker A to other Broker B, C,D), we still need face the scenario which is described by [~Json Tu]: "if there are some partition leader did not change in this request but not have enough replica, then it will not satisfy such code as below." if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) forceComplete() > When the broker graceful shutdown occurs, the producer side sends timeout. > -------------------------------------------------------------------------- > > Key: KAFKA-5678 > URL: https://issues.apache.org/jira/browse/KAFKA-5678 > Project: Kafka > Issue Type: Improvement > Affects Versions: 0.9.0.0, 0.10.0.0, 0.11.0.0 > Reporter: tuyang > > Test environment as follows. > 1.Kafka version:0.9.0.1 > 2.Cluster with 3 broker which with broker id A,B,C > 3.Topic with 6 partitions with 2 replicas,with 2 leader partitions at each > broker. > We can reproduce the problem as follows. > 1.we send message as quickly as possible with ack -1. > 2.if partition p0's leader is on broker A and we graceful shutdown broker > A,but we send a message to p0 before the leader is reelect, so the message > can be appended to the leader replica successful, but if the follower replica > not catch it as quickly as possible, so the shutting down broker will create > a delayProduce for this request to wait complete until request.timeout.ms . > 3.because of the controllerShutdown request from broker A, then the p0 > partition leader will reelect > , then the replica on broker A will become follower before complete shut > down.then the delayProduce will not be trigger to complete until expire. > 4.if broker A shutdown cost too long, then the producer will get response > after request.timeout.ms, which results in increase the producer send latency > when we are restarting broker one by one. -- This message was sent by Atlassian JIRA (v6.4.14#64029)