[ 
https://issues.apache.org/jira/browse/KAFKA-10107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17129077#comment-17129077
 ] 

William Reynolds commented on KAFKA-10107:
------------------------------------------

Still working on getting the logs out, apologies for delay

> Producer snapshots LSO used in certain situations which can lead to data loss 
> on compacted topics as LSO breach occurs and early offsets cleaned
> ------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-10107
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10107
>             Project: Kafka
>          Issue Type: Bug
>          Components: core, log cleaner
>    Affects Versions: 2.4.1
>         Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
> Kafka 2.4.1 on jre 11 on debian 9 in docker
>            Reporter: William Reynolds
>            Priority: Major
>
> While upgading a 1.1.0 cluster to 2.4.1 and also adding an interbroker port 
> using SSL we ran into a situation where producer snapshot offsets get set as 
> the log start offset then logs truncate to nothing across 2 relatively unsafe 
> restarts.
>  
> Here is the timeline of what we did to trigger this
> Broker 40 is shutdown as first to go to 2.4.1 and switch to interbroker port 
> 9094.
>  As it shuts down it writes producer snapshots
>  Broker 40 starts on 2.4.1, loads the snapshots then compares checkpointed 
> offsets to log start offset and finds them to be invalid (exact reason 
> unknown but looks to be producer snapshot load related)
>  On broker 40 all topics show an offset reset like this 2020-05-18 
> 15:22:21,106] WARN Resetting first dirty offset of topic-name-60 to log start 
> offset 6009368 since the checkpointed offset 5952382 is invalid. 
> (kafka.log.LogCleanerManager$)" which then triggers log cleanup on broker 40 
> for all these topics which is where the data is lost
>  At this point only partitions led by broker 40 have lost data and would be 
> failing for client lookups on older data but this can't spread as 40 has 
> interbroker port 9094 and brokers 50 and 60 have interbroker port 9092
>  I stop start brokers 50 and 60 in quick succession to take them to 2.4.1 and 
> onto the new interbroker port 9094
>  This leaves broker 40 as the in sync replica for all but a couple of 
> partitions which aren't on 40 at all shown in the attached image
>  Brokers 50 and 60 start and then take their start offset from leader (or if 
> there was no leader pulls from recovery on returning broker 50 or 60) and so 
> all the replicas also clean logs to remove data to catch up to broker 40 as 
> that is the in sync replica
>  Then I shutdown 40 and 50 leading to 60 leading all partitions it holds and 
> then we see this happen across all of those partitions
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,251] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Loading producer 
> state till offset 0 with message format version 2 (kafka.log.Log)" 
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,252] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Completed load of 
> log with 1 segments, log start offset 0 and log end offset 0 in 2 ms 
> (kafka.log.Log)"
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] WARN [ReplicaFetcher replicaId=50, leaderId=60, fetcherId=0] 
> Leader or replica is on protocol version where leader epoch is not considered 
> in the OffsetsForLeaderEpoch response. The leader's offset 0 will be used for 
> truncation in topic-name-60. (kafka.server.ReplicaFetcherThread)" 
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] 
> Truncating to offset 0 (kafka.log.Log)"
>  
> I believe the truncation has always been a problem but recent 
> https://issues.apache.org/jira/browse/KAFKA-6266 fix allowed truncation to 
> actually happen where it wouldn't have before. 
>  The producer snapshots setting as log start offset is a mystery to me so any 
> light you could shed on why that yhappened and how to avoid would be great.
>  
> I am sanitising full logs and will upload here soon



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to