The scenario you mentioned is equivalent to an unclean leader election.
The following settings will make sure there is no data loss:
1. Set replica factor to 3 and minimum ISR size to 2.
2. When produce, use acks=-1 or acks=all
3. Disable unclean leader election.

1) and 2) Guarantees committed messages will be at least in to brokers.
3) Means if a broker is not in ISR, it cannot be elected as a leader, so
the log truncate as mentioned earlier will not happen.

Jiangjie (Becket) Qin

On 3/2/15, 7:16 PM, "tao xiao" <xiaotao...@gmail.com> wrote:

>Since I reused the same consumer group to consume the messages after step
>6
>data there was no data loss occurred. But if I create a new consumer group
>for sure the new consumer will suffer data loss.
>
>I am more concerning about if this is an acceptable behavior by Kafka that
>an out of sync broker can be elected as the leader for a partition. Is
>there any mechanism built around Kafka to ensure that only the in-sync
>broker can be chosen to be a leader? If no, what is the best practice to
>restart brokers if some of the replicas are out of sync?
>
>On Tue, Mar 3, 2015 at 2:35 AM, Jiangjie Qin <j...@linkedin.com.invalid>
>wrote:
>
>> In this case you have data loss. In step 6, when broker 1 comes up, it
>> becomes the leader and has log end offset 1000. When broker 0 comes up,
>>it
>> becomes follower and will truncate its log to 1000, i.e. 1000 messages
>> were lost. Next time when the consumer starts, its offset will be reset
>>to
>> either the smallest or the largest depending on the setting.
>>
>> Jiangjie (Becket) Qin
>>
>> On 3/2/15, 9:32 AM, "Stuart Reynolds" <s...@stureynolds.com> wrote:
>>
>> >Each topic has:  earliest and latest offsets (per partition)
>> >Each consumer group has a current offset (per topic, partition pair)
>> >
>> >I see -1 for the current offsets new consumer groups that haven't yet
>> >committed an offset. I think it means that the offsets for that
>> >consumer group are undefined.
>> >
>> >Is it possible you generated new consumer groups when you restarted
>>your
>> >broker?
>> >
>> >
>> >
>> >
>> >On Mon, Mar 2, 2015 at 3:15 AM, tao xiao <xiaotao...@gmail.com> wrote:
>> >> Hi team,
>> >>
>> >> I have 2 brokers (0 and 1) serving a topic mm-benchmark-test. I did
>>some
>> >> tests on the two brokers to verify how leader got elected. Here are
>>the
>> >> steps:
>> >>
>> >> 1. started 2 brokers
>> >> 2. created a topic with partition=1 and replication-factor=2. Now
>> >>brokers 1
>> >> was elected as leader
>> >> 3. sent 1000 messages to the topic and consumed from a high level
>> >>consumer
>> >> using zk as the offset storage.
>> >> 4. shutdown broker 1 and now broker 0 was elected as leader
>> >> 5. sent another 1000 messages to topic and consumed again
>> >> 6. completely shutdown broker 0 and then started broker 1. now
>>broker 1
>> >> became the leader
>> >> 7. started broker 0 and ran ConsumerOffsetChecker which showed
>>negative
>> >>lag
>> >> (-1000 in my case)
>> >>
>> >> I think this is because the consumed offset in zk was 2000 and
>>logsize
>> >> retrieved from the leader (broker 1) which missed 1000 messages in
>>step
>> >>5
>> >> in this case was 1000 there -1000 = 1000 - 2000 was given.
>> >>
>> >> Is this a bug or expected behavior?
>> >>
>> >> --
>> >> Regards,
>> >> Tao
>>
>>
>
>
>-- 
>Regards,
>Tao

Reply via email to