acks = all should prevent this scenario:

If broker 0 is still in ISR, the produce request for 101 will not be
"acked" (because 0 is in ISR and not available for acking), and the
producer will retry it until all ISR acks.

If broker 0 dropped off ISR, it will not be able to rejoin until it has all
the latest messages, including 101.

So if you use the safe settings you should be safe in this scenario.

Gwen


On Sat, Sep 12, 2015 at 3:15 AM, Zhao Weinan <zhaow...@gmail.com> wrote:

> Hi group,
>
> I've been through some data loss on kafka cluster, one case maybe caused by
> the auto preferred leader election.
>
> Here is the situation: 3 brokers = {0,1,2}, 1 partition with 3 replicas on
> 0/1/2, all in sync while 0 is leader and 1 is controller, current offset is
> 100.
>
> And here is my hypothesis:
> a. Leader 0 is temporary gone due to instablity connection with ZK
> b. Controller 1 found that 0 has gone then do a election which let 1(in
> ISR) to be leader
> c. A producer send 1 message to new leader 1, then offset is 101
> d. Old leader 0 is back to cluster(*STILL IN ISR* because the lag not
> exceed the *replica.lag.time.max.ms <http://replica.lag.time.max.ms>* and
> *replica.lag.max.messages*)
> e. Coincidentally, controller 1 start to checkAndTriggerPartitionRebalance
> then decide 0 is more preferred, so let 0 to be back to leader
> f. Broker 1 become to follower then found HW to be 100, so truncate to 100
> that lead to lost newest message.
>
> With this situation, using most reliable settings(broker side:
> unclean.leader.election.enable=false, min.insync.replicas=2; producer side
> acks=all) is useless. Am I correct or just paranoia?
>
> Below is some real logs from production.
> I*n controller.log:*
>
> > *// broker 6 tempraroy gone*
> >
> [2015-09-09 15:24:42,206] INFO [BrokerChangeListener on Controller 3]:
> > Newly added brokers: , deleted brokers: 6, all live brokers:
> 0,5,1,2,7,3,4
> > (kafka.controller.ReplicaStateMachine$BrokerChangeListener)
> > [2015-09-09 15:24:42,461] INFO [Controller 3]: Broker failure callback
> for
> > 6 (kafka.controller.KafkaController)
> > [2015-09-09 15:24:42,464] INFO [Controller 3]: Removed ArrayBuffer() from
> > list of shutting down brokers. (kafka.controller.KafkaController)
> > [2015-09-09 15:24:42,466] INFO [Partition state machine on Controller 3]:
> > Invoking state change to OfflinePartition for partitions
> > [SOME_TOPIC_NAME,1] (kafka.controller.PartitionStateMachine)
> >
> > *// elect 3 which in ISR to be leader*
> > [2015-09-09 15:24:43,182] DEBUG [OfflinePartitionLeaderSelector]: Some
> > broker in ISR is alive for [SOME_TOPIC_NAME,1]. Select 3 from ISR 3,4 to
> be
> > the leader. (kafka.controller.OfflinePartitionLeaderSelector)
> > [2015-09-09 15:24:43,182] INFO [OfflinePartitionLeaderSelector]: Selected
> > new leader and ISR {"leader":3,"leader_epoch":45,"isr":[3,4]} for offline
> > partition [SOME_TOPIC_NAME,1]
> > (kafka.controller.OfflinePartitionLeaderSelector)
> > [2015-09-09 15:24:43,928] DEBUG [Controller 3]: Removing replica 6 from
> > ISR 3,4 for partition [SOME_TOPIC_NAME,1].
> > (kafka.controller.KafkaController)
> > [2015-09-09 15:24:43,929] WARN [Controller 3]: Cannot remove replica 6
> > from ISR of partition [SOME_TOPIC_NAME,1] since it is not in the ISR.
> > Leader = 3 ; ISR = List(3, 4) (kafka.controller.KafkaController)
> >
> > *// broker 6 back*
> >
> [2015-09-09 15:24:44,575] INFO [BrokerChangeListener on Controller 3]:
> > Newly added brokers: 6, deleted brokers: , all live brokers:
> > 0,5,1,6,2,7,3,4
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener)
> >
> > *// broker 6 is elected as leader by auto preferred leader election*
> >
> [2015-09-09 15:24:50,939] INFO [Controller 3]: Starting preferred replica
> > leader election for partitions [SOME_TOPIC_NAME,1]
> > (kafka.controller.KafkaController)
> > [2015-09-09 15:24:50,945] INFO [PreferredReplicaPartitionLeaderSelector]:
> > Current leader 3 for partition [SOME_TOPIC_NAME,1] is not the preferred
> > replica. Trigerring preferred replica leader election
> > (kafka.controller.PreferredReplicaPartitionLeaderSelector)
> >
> >
> *And in server.log:*
>
> > *// broker 3 truncating, lossing data*
>
> 2015-09-09 15:24:50,964] INFO Truncating log SOME_TOPIC_NAME-1 to offset
> > 420549. (kafka.log.Log)
> >
>

Reply via email to