Looks like the screen shots didn't come through. 
Consider pasting the text.
Thanks
-------- Original message --------From: Yogesh Sangvikar 
<yogesh.sangvi...@gmail.com> Date: 9/19/17  4:33 AM  (GMT-08:00) To: 
users@kafka.apache.org Cc: Sumit Arora <sumit.ar...@ascendlearning.com>, 
Bharathreddy Sodinapalle <bharathredd...@happiestminds.com>, 
asgar....@happiestminds.com Subject: Re: Data loss while upgrading confluent 
3.0.0 kafka cluster to confluent 3.2.2 
Hi Team,

Thanks for providing comments.

Here adding more details on steps followed for upgrade,
Cluster details: We are using 4 node kafka cluster and topics with 3 
replication factor. For upgrade test, we are using a topic with 5 partitions & 
3 replication factor.
Topic:student-activity  PartitionCount:5        ReplicationFactor:3     
Configs:        Topic: student-activity Partition: 0    Leader: 4       
Replicas: 4,2,3 Isr: 4,2,3        Topic: student-activity Partition: 1    
Leader: 1       Replicas: 1,3,4 Isr: 1,4,3        Topic: student-activity 
Partition: 2    Leader: 2       Replicas: 2,4,1 Isr: 2,4,1        Topic: 
student-activity Partition: 3    Leader: 3       Replicas: 3,1,2 Isr: 1,2,3     
   Topic: student-activity Partition: 4    Leader: 4       Replicas: 4,3,1 Isr: 
4,1,3
We are using a test script to publish events continuously to one of the topic 
partition (here partition 3) and monitoring the scripts total published events 
count  with the partition 3 offset value.

[ Note: The topic partitions offset count may differ from CLI utility and 
screenshot due to capture delay. ]First, we have rolling restarted all kafka 
brokers for explicit protocol and message version to 0.10.0, 
inter.broker.protocol.version=0.10.0 
log.message.format.version=0.10.0

During this restarted, the events are getting published as expected and 
counters are increasing & in-sync replicas are coming up immediately post 
restart.

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:785
student-activity:0:1



Next, we  have rolling restarted kafka brokers for 
"inter.broker.protocol.version=0.10.2" in below broker sequence. (note that, 
test script is publishing events to the topic partition continuously)
 Restarted server with  broker.id = 4,

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:1189
student-activity:0:1



Restarted server with  broker.id = 3,

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:1430
student-activity:0:1
          

Restarted server with  broker.id = 2, (here, observe the partition 3 offset 
count is decreased from last restart offset)

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:1357
student-activity:0:1
           
Restarted last server with  broker.id = 1,

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:1613
student-activity:0:1


Finally, rolling restarted all brokers (in same sequence above) for 
"log.message.format.version=0.10.2"

​ 




The topic offset counter after final restart,

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:2694
student-activity:0:1


And, the topic offset counter after stopping events publish script,

[***@***.***.***.*** confluent-3.2.2]$ ./bin/kafka-run-class 
kafka.tools.GetOffsetShell --broker-list 
***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092,***.***.***.***:9092
 --topic student-activity --time -1
student-activity:2:1
student-activity:4:1
student-activity:1:68
student-activity:3:2769
student-activity:0:1

Calculating missing events counts,
Total events published by script to partition 3 : 3090
Offset count on Partition 3 : 2769

Missing events count : 3090 - 2769 = 321
 As per above observation during rolling restart for protocol version, The 
partition 3 leader changed to in-sync replica 2 (with older protocol version) 
and upgraded replicas (3 & 4) are missing from in-sync replica list.And, one we 
down server 2 down for upgrade, suddenly replicas 3 & 4 appear in in-sync 
replica list and partition offset count resets.Post server 2 & 1 upgrade, 3 
in-sync replicas shown for partition 3 but, missing events lag is not 
recovered.    
Please let us know your comments on our observations and correct us if we are 
missing any upgrade steps.

Thanks,Yogesh  
On Tue, Sep 19, 2017 at 2:07 AM, Ismael Juma <ism...@juma.me.uk> wrote:
Hi Scott,



There is nothing preventing a replica running a newer version from being in

sync as long as the instructions are followed (i.e.

inter.broker.protocol.version has to be set correctly and, if there's a

message format change, log.message.format.version). That's why I asked

Yogesh for more details. The upgrade path he mentioned (0.10.0 -> 0.10.2)

is straightforward, there isn't a message format change, so only

inter.broker.protocol.version needs to be set.



Ismael



On Mon, Sep 18, 2017 at 5:50 PM, Scott Reynolds <

sreyno...@twilio.com.invalid> wrote:



> Can we get some clarity on this point:

> >older version leader is not allowing newer version replicas to be in sync,

> so the data pushed using this older version leader

>

> That is super scary.

>

> What protocol version is the older version leader running?

>

> Would this happen if you are skipping a protocol version bump?

>

>

>

> On Mon, Sep 18, 2017 at 9:33 AM Ismael Juma <ism...@juma.me.uk> wrote:

>

> > Hi Yogesh,

> >

> > Can you please clarify what you mean by "observing data loss"?

> >

> > Ismael

> >

> > On Mon, Sep 18, 2017 at 5:08 PM, Yogesh Sangvikar <

> > yogesh.sangvi...@gmail.com> wrote:

> >

> > > Hi Team,

> > >

> > > Please help to find resolution for below kafka rolling upgrade issue.

> > >

> > > Thanks,

> > >

> > > Yogesh

> > >

> > > On Monday, September 18, 2017 at 9:03:04 PM UTC+5:30, Yogesh Sangvikar

> > > wrote:

> > >>

> > >> Hi Team,

> > >>

> > >> Currently, we are using confluent 3.0.0 kafka cluster in our

> production

> > >> environment. And, we are planing to upgrade the kafka cluster for

> > confluent

> > >> 3.2.2

> > >> We are having topics with millions on records and data getting

> > >> continuously published to those topics. And, also, we are using other

> > >> confluent services like schema-registry, kafka connect and kafka rest

> to

> > >> process the data.

> > >>

> > >> So, we can't afford downtime upgrade for the platform.

> > >>

> > >> We have tries rolling kafka upgrade as suggested on blogs in

> Development

> > >> environment,

> > >>

> > >>

> > https://urldefense.proofpoint.com/v2/url?u=https-3A__docs.

> confluent.io_3.2.2_upgrade.html&d=DwIBaQ&c=x_Y1Lz9GyeGp2OvBCa_eow&r=

> ChXZJWKniTslJvQGptpIW7qAh4kkrpgYSer_wfh4G5w&m=JGTnLlVIAvVddNas19L_

> w54zWrVd48xst46GuPGCxV0&s=DMcA8JOnGXNNa_dRFpkNOd7AJoIQUgkEcw6q06RHgl0&e=

> > >>

> > >>

> > https://urldefense.proofpoint.com/v2/url?u=https-3A__kafka.

> apache.org_documentation_-23upgrade&d=DwIBaQ&c=x_Y1Lz9GyeGp2OvBCa_eow&r=

> ChXZJWKniTslJvQGptpIW7qAh4kkrpgYSer_wfh4G5w&m=JGTnLlVIAvVddNas19L_

> w54zWrVd48xst46GuPGCxV0&s=0p4Fn8sKMbVJMR6nk42C-lhyujAEVUXTYZJhteC11Fs&e=

> > >>

> > >> But, we are observing data loss on topics while doing rolling upgrade

> /

> > >> restart of kafka servers for "inter.broker.protocol.version=0.10.2".

> > >>

> > >> As per our observation, we suspect the root cause for the data loss

> > >> (explained for a topic partition having 3 replicas),

> > >>

> > >>    - As the kafka broker protocol version updates from 0.10.0 to

> 0.10.2

> > >>    in rolling fashion, the in-sync replicas having older version will

> > not

> > >>    allow updated replicas (0.10.2) to be in sync unless are all

> updated.

> > >>    - Also, we have explicitly disabled "unclean.leader.election.

> enabled"

> > >>    property, so only in-sync replicas will be elected as leader for

> the

> > given

> > >>    partition.

> > >>    - While doing rolling fashion update, as mentioned above, older

> > >>    version leader is not allowing newer version replicas to be in

> sync,

> > so the

> > >>    data pushed using this older version leader, will not be synced

> with

> > other

> > >>    replicas and if this leader(older version)  goes down for an

> > upgrade, other

> > >>    updated replicas will be shown in in-sync column and become leader,

> > but

> > >>    they lag in offset with old version leader and shows the offset of

> > the data

> > >>    till they have synced.

> > >>    - And, once the last replica comes up with updated version, will

> > >>    start syncing data from the current leader.

> > >>

> > >>

> > >> Please let us know comments on our observation and suggest proper way

> > for

> > >> rolling kafka upgrade as we can't afford downtime.

> > >>

> > >> Thanks,

> > >> Yogesh

> > >>

> > >

> >

> --

>

> Scott Reynolds

> Principal Engineer

> [image: twilio] <http://www.twilio.com/?utm_source=email_signature>

> MOBILE (630) 254-2474

> EMAIL sreyno...@twilio.com

>



Reply via email to