You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "John Fung (JIRA)" <ji...@apache.org> on 2012/10/05 01:31:47 UTC

[jira] [Commented] (KAFKA-514) Replication with Leader Failure Test: Log segment files checksum mismatch

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

John Fung commented on KAFKA-514:
---------------------------------

This issue can be reproduced as follows:

1. Download the latest 0.8 branch
2. Apply kafka-502-v4.patch
3. Under directory <kafka_home>, execute "./sbt update package" to build Kafka
4. Untar testcase_2.tar to <kafka_home>/system_test/replication_testsuite/
5. Modified <kafka_home>/system_test/testcase_to_run.json from "testcase_1" to "testcase_2"
6. Under directory <kafka_home>/system_test, execute "python -B system_test_runner.py"
                
> Replication with Leader Failure Test: Log segment files checksum mismatch
> -------------------------------------------------------------------------
>
>                 Key: KAFKA-514
>                 URL: https://issues.apache.org/jira/browse/KAFKA-514
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8
>            Reporter: John Fung
>            Priority: Blocker
>              Labels: replication-testing
>             Fix For: 0.8
>
>         Attachments: system_test_output_archive.tar.gz, testcase_2.tar
>
>
> Test Description:
>    1. Produce and consume messages to 1 topics and 3 partitions.
>    2. This test sends 10 messages every 2 sec to 3 replicas.
>    3. At the end verifies the log size and contents as well as using a consumer to verify that there is no message loss.
> The issue:
> When the leader is terminated by a controlled failure (kill -15), the resulting log segment files size are not all matching. The mismatch log segment size would happen in one of the partition of the terminated broker. This is consistently reproducible from the system regression test for replication with the following configurations:
>     * zookeeper: 1-node (local)
>     * brokers: 3-node cluster (all local)
>     * replica factor: 3
>     * no. of topic: 1
>     * no. of partition: 2
>     * iterations of leader failure: 1
> Remarks:
>     * It is rarely reproducible if the no. of partitions is 1.
>     * Even the file checksums are not matching, the no. of messages in the producer & consumer logs are equal
> Test result (shown with log file checksum):
> broker-1 :
> test_1-0/00000000000000000000.kafka => 1690639555
> test_1-1/00000000000000000000.kafka => 4068655384    <<<< not matching across all replicas
> broker-2 :
> test_1-0/00000000000000000000.kafka => 1690639555
> test_1-1/00000000000000000000.kafka => 4068655384    <<<< not matching across all replicas
> broker-3 :
> test_1-0/00000000000000000000.kafka => 1690639555
> test_1-1/00000000000000000000.kafka => 3530842923    <<<< not matching across all replicas
> Errors:
> The following error is found in the terminated leader:
> [2012-09-14 11:07:05,217] WARN No previously checkpointed highwatermark value found for topic test_1 partition 1. Returning 0 as the highwatermark (kafka.server.HighwaterMarkCheckpoint)
> [2012-09-14 11:07:05,220] ERROR Replica Manager on Broker 3: Error processing leaderAndISR request LeaderAndIsrRequest(1,,true,1000,Map((test_1,1) -> { "ISR": "1,2","leader": "1","leaderEpoch": "0" }, (test_1,0) -> { "ISR": "
> 1,2","leader": "1","leaderEpoch": "1" })) (kafka.server.ReplicaManager)
> kafka.common.KafkaException: End index must be segment list size - 1
>         at kafka.log.SegmentList.truncLast(SegmentList.scala:82)
>         at kafka.log.Log.truncateTo(Log.scala:471)
>         at kafka.cluster.Partition.makeFollower(Partition.scala:171)
>         at kafka.cluster.Partition.makeLeaderOrFollower(Partition.scala:126)
>         at kafka.server.ReplicaManager.kafka$server$ReplicaManager$$makeFollower(ReplicaManager.scala:195)
>         at kafka.server.ReplicaManager$$anonfun$becomeLeaderOrFollower$2.apply(ReplicaManager.scala:154)
>         at kafka.server.ReplicaManager$$anonfun$becomeLeaderOrFollower$2.apply(ReplicaManager.scala:144)
>         at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:80)
>         at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:80)
>         at scala.collection.Iterator$class.foreach(Iterator.scala:631)
>         at scala.collection.mutable.HashTable$$anon$1.foreach(HashTable.scala:161)
>         at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:194)
>         at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
>         at scala.collection.mutable.HashMap.foreach(HashMap.scala:80)
>         at kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:144)
>         at kafka.server.KafkaApis.handleLeaderAndISRRequest(KafkaApis.scala:73)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
>         at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:40)
>         at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira