Thanks for the tips. As far as a broker being down, the situation is confused because a broker was definitely down, but I think we brought it down when we were trying to address the problem a few days ago. I can't say for sure that it wasn't down before, but we do have a monitor to catch that (by trying to telnet to the broker) and it was not firing on the 14th when things went sideways.
I looked in server.logs and found many instances of this exception at the time things went bad: org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. Also some like "error when handling request Name:UpdateMetadataRequest" There's a few different sorts of errors in some of the different server.log but they all seem to be varieties of this one. We have 5 nodes - is there one in particular where I should be checking the logs for errors, or any particular error to look for that would be a likely root cause? -----Original Message----- From: Swapnil Gupta [mailto:neomatrix1...@gmail.com] Sent: Friday, March 23, 2018 8:00 AM To: users@kafka.apache.org Subject: Re: can't consume from partitions due to KAFKA-3963 Maybe a broker is down or unreachable which maybe breaking your min isr ratio and when consumers are set to ack all, the min isr has to be satisfied. Check your broker connect, or bring up a fresh broker and use preferred replica leader election tool -> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools to re balance with the existing data with the new broker. Check which node is down, check isr, under replication and then you get the node id which maybe causing the trouble. And better check *server.log* files and grep for the error and caused by over there, that will provide you with exact reason. Regards, Swap On Fri, Mar 23, 2018 at 1:50 AM, Ryan O'Rourke <rorou...@marchex.com> wrote: > Hi, we're having an outage in our production Kafka and getting > desperate, any help would be appreciated. > > On 3/14 our consumer (a Storm spout) started getting messages from > only 20 out of 40 partitions on a topic. We only noticed yesterday. > Restarting the consumer with a new consumer group does not fix the problem. > > We just found some errors in the Kafka state change log which look > like they may be related - the example is definitely one of the > affected partition, and the timestamp lines up with when the problem > started. Seems to be related to KAFKA-3963. What can we do to mitigate > this and prevent it from happening again? > > kafka.common.NoReplicaOnlineException: No replica for partition > [transcription-results,9] is alive. Live brokers are: [Set()], > Assigned replicas are: [List(1, 4, 0)] > [2018-03-14 03:11:40,863] TRACE Controller 0 epoch 44 changed state of > replica 1 for partition [transcription-results,9] from OnlineReplica > to OfflineReplica (state.change.logger) > [2018-03-14 03:11:41,141] TRACE Controller 0 epoch 44 sending > become-follower LeaderAndIsr request > (Leader:-1,ISR:0,4,LeaderEpoch:442,ControllerEpoch:44) > to broker 4 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,145] TRACE Controller 0 epoch 44 sending > become-follower LeaderAndIsr request > (Leader:-1,ISR:0,4,LeaderEpoch:442,ControllerEpoch:44) > to broker 0 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,208] TRACE Controller 0 epoch 44 changed state of > replica 4 for partition [transcription-results,9] from OnlineReplica > to OnlineReplica (state.change.logger) > [2018-03-14 03:11:41,218] TRACE Controller 0 epoch 44 changed state of > replica 1 for partition [transcription-results,9] from OfflineReplica > to OnlineReplica (state.change.logger) > [2018-03-14 03:11:41,226] TRACE Controller 0 epoch 44 sending > become-follower LeaderAndIsr request > (Leader:-1,ISR:0,4,LeaderEpoch:442,ControllerEpoch:44) > to broker 4 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,230] TRACE Controller 0 epoch 44 sending > become-follower LeaderAndIsr request > (Leader:-1,ISR:0,4,LeaderEpoch:442,ControllerEpoch:44) > to broker 1 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,450] TRACE Broker 0 received LeaderAndIsr request > (LeaderAndIsrInfo:(Leader:-1,ISR:0,4,LeaderEpoch:442,ControllerEpoch:4 > 4), > ReplicationFactor:3),AllReplicas:1,4,0) correlation id 158 from > controller 0 epoch 44 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,454] TRACE Broker 0 handling LeaderAndIsr request > correlationId 158 from controller 0 epoch 44 starting the > become-follower transition for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,455] ERROR Broker 0 received LeaderAndIsrRequest > with correlation id 158 from controller 0 epoch 44 for partition > [transcription-results,9] but cannot become follower since the new > leader > -1 is unavailable. (state.change.logger) > [2018-03-14 03:11:41,459] TRACE Broker 0 completed LeaderAndIsr > request correlationId 158 from controller 0 epoch 44 for the > become-follower transition for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,682] TRACE Controller 0 epoch 44 started leader > election for partition [transcription-results,9] (state.change.logger) > [2018-03-14 03:11:41,687] TRACE Controller 0 epoch 44 elected leader 4 > for Offline partition [transcription-results,9] (state.change.logger) > [2018-03-14 03:11:41,689] TRACE Controller 0 epoch 44 changed > partition [transcription-results,9] from OfflinePartition to > OnlinePartition with leader 4 (state.change.logger) > [2018-03-14 03:11:41,825] TRACE Controller 0 epoch 44 sending > become-leader LeaderAndIsr request > (Leader:4,ISR:4,LeaderEpoch:443,ControllerEpoch:44) > to broker 4 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,826] TRACE Controller 0 epoch 44 sending > become-follower LeaderAndIsr request > (Leader:4,ISR:4,LeaderEpoch:443,ControllerEpoch:44) > to broker 1 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,899] TRACE Broker 0 cached leader info > (LeaderAndIsrInfo:(Leader:1,ISR:0,1,4,LeaderEpoch:441,ControllerEpoch: > 44), > ReplicationFactor:3),AllReplicas:1,4,0) for partition > [transcription-results,9] in response to UpdateMetadata request sent > by controller 1 epoch 47 with correlation id 0 (state.change.logger) > [2018-03-14 03:11:41,906] TRACE Broker 0 received LeaderAndIsr request > (LeaderAndIsrInfo:(Leader:1,ISR:0,1,4,LeaderEpoch:441,ControllerEpoch: > 44), > ReplicationFactor:3),AllReplicas:1,4,0) correlation id 1 from > controller > 1 epoch 47 for partition [transcription-results,9] > (state.change.logger) > [2018-03-14 03:11:41,908] WARN Broker 0 ignoring LeaderAndIsr request > from controller 1 with correlation id 1 epoch 47 for partition > [transcription-results,9] since its associated leader epoch 441 is old. > Current leader epoch is 441 (state.change.logger) > [2018-03-14 03:11:41,982] TRACE Broker 0 cached leader info > (LeaderAndIsrInfo:(Leader:1,ISR:0,1,4,LeaderEpoch:441,ControllerEpoch: > 44), > ReplicationFactor:3),AllReplicas:1,4,0) for partition > [transcription-results,9] in response to UpdateMetadata request sent > by controller 1 epoch 47 with correlation id 2 (state.change.logger) > [2018-03-22 14:43:36,098] TRACE Broker 0 received LeaderAndIsr request > (LeaderAndIsrInfo:(Leader:-1,ISR:,LeaderEpoch:444,ControllerEpoch:47), > ReplicationFactor:3),AllReplicas:1,4,0) correlation id 679 from > controller 1 epoch 47 for partition [transcription-results,9] > (state.change.logger) > > > -- <https://in.linkedin.com/in/swappy>