[ https://issues.apache.org/jira/browse/KAFKA-607?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13495024#comment-13495024 ]
John Fung commented on KAFKA-607: --------------------------------- • This issue is transient and is showing randomly in this functional test group. The test case is failing because ConsoleConsumer is receiving less data than producer produced. • Transient Failure Testcase: 4011 • Functional Test Group: Log Retention (Size) / 2 topics / 2 partitions / Replica Factor = 2 / No. of brokers = 3 / One Leader bouncing Unique messages from consumer on [test_1] : 500 Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r1.log : 500 Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r2.log : 0 Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r3.log : 500 Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r1.log : 500 Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r2.log : 500 Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r3.log : 0 Unique messages from consumer on [test_2] : 295 Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r1.log : 500 Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r2.log : 0 Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r3.log : 500 Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r1.log : 500 Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r2.log : 500 Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r3.log : 0 Unique messages from producer on [test_1] : 1000 Unique messages from producer on [test_2] : 1000 Validate for data matched on topic [test_1] : FAILED Validate for data matched on topic [test_1] across replicas : PASSED Validate for data matched on topic [test_2] : FAILED Validate for data matched on topic [test_2] across replicas : PASSED Validate leader election successful : PASSED • By checking the broker and ConsoleConsumer log4j messages, it appears that when leader is stopped, ConsoleConsumer is disconnected from Zookeeper. • Broker log4j messages: 2012-11-10 04:52:52,857 - INFO - found leader in entity [3] with brokerid [3] for partition [0] (kafka_system_test_utils) 2012-11-10 04:52:52,857 - INFO - stopping leader in entity 3 with pid 16146 (kafka_system_test_utils) 2012-11-10 04:52:53,036 - DEBUG - executing command [ssh ela4-app0999.prod 'kill -15 16148'] (system_test_utils) 2012-11-10 04:52:53,055 - DEBUG - executing command [ssh ela4-app0999.prod 'kill -15 16146'] (system_test_utils) • Log4j messages in the Broker that was bounced: [2012-11-10 04:52:53,200] INFO Replica Manager on Broker 3: Shutted down completely (kafka.server.ReplicaManager) [2012-11-10 04:52:53,201] INFO [Socket Server on Broker 3], shutting down (kafka.network.SocketServer) [2012-11-10 04:52:53,205] INFO [Socket Server on Broker 3], shutted down completely (kafka.network.SocketServer) [2012-11-10 04:52:53,212] INFO [Kafka Server 3], shut down completed (kafka.server.KafkaServer) [2012-11-10 04:53:09,497] INFO Started Kafka CSV metrics reporter with polling period 5 seconds (kafka.metrics.KafkaCSVMetricsReporter) [2012-11-10 04:53:09,506] INFO [Kafka Server 3], starting (kafka.server.KafkaServer) [2012-11-10 04:53:09,535] INFO [Log Manager on Broker 3] Loading log 'test_2-0' (kafka.log.LogManager) [2012-11-10 04:53:09,579] INFO [Log Manager on Broker 3] Loading log 'test_1-0' (kafka.log.LogManager) [2012-11-10 04:53:09,583] INFO [Log Manager on Broker 3] Starting log cleaner every 60000 ms (kafka.log.LogManager) [2012-11-10 04:53:09,588] INFO [Log Manager on Broker 3] Starting log flusher every 1000 ms with the following overrides Map() (kafka.log.LogManager) . . . [2012-11-10 04:53:09,692] INFO zookeeper state changed (SyncConnected) (org.I0Itec.zkclient.ZkClient) [2012-11-10 04:53:09,694] INFO Registering broker /brokers/ids/3 (kafka.server.KafkaZooKeeper) [2012-11-10 04:53:09,705] INFO Registering broker /brokers/ids/3 succeeded with id:3,creatorId:172.17.166.125-1352523189694,host:172.17.166.125,port:9093 (kafka.utils.ZkUtils$) [2012-11-10 04:53:09,705] INFO [Kafka Server 3], Connecting to ZK: ela4-app0996.prod:2188 (kafka.server.KafkaServer) [2012-11-10 04:53:09,772] INFO Will not load MX4J, mx4j-tools.jar is not in the classpath (kafka.utils.Mx4jLoader$) [2012-11-10 04:53:09,774] INFO [Controller 3]: Controller starting up (kafka.controller.KafkaController) [2012-11-10 04:53:09,781] ERROR [KafkaApi-3] error when processing request (test_2,0,150,1048576) (kafka.server.KafkaApis) kafka.common.UnknownTopicOrPartitionException: Topic test_2 partition 0 doesn't exist on 3 at kafka.server.ReplicaManager.getLeaderReplicaIfLocal(ReplicaManager.scala:142) at kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSet(KafkaApis.scala:363) at kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:329) at kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:325) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) at scala.collection.immutable.Map$Map1.foreach(Map.scala:105) at scala.collection.TraversableLike$class.map(TraversableLike.scala:206) at scala.collection.immutable.Map$Map1.map(Map.scala:93) at kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSets(KafkaApis.scala:325) at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:293) at kafka.server.KafkaApis.handle(KafkaApis.scala:58) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:41) at java.lang.Thread.run(Thread.java:619)[2012-11-10 04:53:09,781] ERROR [KafkaApi-3] error when processing request (test_1,0,250,1048576) (kafka.server.KafkaApis) kafka.common.UnknownTopicOrPartitionException: Topic test_1 partition 0 doesn't exist on 3 at kafka.server.ReplicaManager.getLeaderReplicaIfLocal(ReplicaManager.scala:142) at kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSet(KafkaApis.scala:363) at kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:329) at kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:325) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) at scala.collection.immutable.Map$Map1.foreach(Map.scala:105) at scala.collection.TraversableLike$class.map(TraversableLike.scala:206) at scala.collection.immutable.Map$Map1.map(Map.scala:93) at kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSets(KafkaApis.scala:325) at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:293) at kafka.server.KafkaApis.handle(KafkaApis.scala:58) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:41) at java.lang.Thread.run(Thread.java:619) [2012-11-10 04:53:09,806] INFO conflict in /controller data: 3 stored data: 1 (kafka.utils.ZkUtils$) [2012-11-10 04:53:09,808] INFO [Controller 3]: Controller startup complete (kafka.controller.KafkaController) [2012-11-10 04:53:09,825] INFO [Kafka Server 3], started (kafka.server.KafkaServer) [2012-11-10 04:53:09,826] INFO Verifying properties (kafka.utils.VerifiableProperties) • ConsoleConsumer log4j messages: [2012-11-10 04:52:53,550] INFO Reconnect due to socket error: (kafka.consumer.SimpleConsumer) java.io.EOFException: Received -1 when reading from channel, socket has likely been closed. at kafka.utils.Utils$.read(Utils.scala:388) at kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54) . . . [2012-11-10 04:52:53,558] INFO Fetching metadata for topic Set(test_1) (kafka.client.ClientUtils$) [2012-11-10 04:52:53,558] INFO Connected to 172.17.166.123:9091 for producing (kafka.producer.SyncProducer) [2012-11-10 04:52:53,582] INFO Disconnecting from 172.17.166.123:9091 (kafka.producer.SyncProducer) [2012-11-10 04:53:36,054] ERROR Error processing message, stopping consumer: (kafka.consumer.ConsoleConsumer$) kafka.consumer.ConsumerTimeoutException at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:67) at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33) . . . [2012-11-10 04:53:36,055] INFO [console-consumer-84076_ela4-app0997.prod-1352523123516-afb2c948], ZKConsumerConnector shutting down (kafka.consumer.ZookeeperConsum erConnector) [2012-11-10 04:53:36,055] INFO Shutting down topic event watcher. (kafka.consumer.ZookeeperTopicEventWatcher) [2012-11-10 04:53:36,055] INFO Terminate ZkClient event thread. (org.I0Itec.zkclient.ZkEventThread)[2012-11-10 04:53:36,057] INFO Session: 0x13ae8a88aae000a closed (org.apache.zookeeper.ZooKeeper) [2012-11-10 04:53:36,057] INFO EventThread shut down (org.apache.zookeeper.ClientCnxn)[2012-11-10 04:53:36,058] INFO Forcing shutdown of Kafka scheduler (kafka.utils.KafkaScheduler) [2012-11-10 04:53:36,059] INFO [ConsumerFetcherManager-1352523123614] shutting down (kafka.consumer.ConsumerFetcherManager) [2012-11-10 04:53:36,059] INFO [console-consumer-84076_ela4-app0997.prod-1352523123516-afb2c948-leader-finder-thread], Shutting down (kafka.consumer.ConsumerFetche rManager$$anon$1) [2012-11-10 04:53:36,703] INFO [console-consumer-84076_ela4-app0997.prod-1352523123516-afb2c948], stopping watcher executor thread for consumer console-consumer-84 076_ela4-app0997.prod-1352523123516-afb2c948 (kafka.consumer.ZookeeperConsumerConnector) [2012-11-10 04:55:02,312] INFO Reconnect due to socket error: (kafka.consumer.SimpleConsumer) java.io.EOFException: Received -1 when reading from channel, socket has likely been closed. at kafka.utils.Utils$.read(Utils.scala:388) at kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54) . . . [2012-11-10 04:55:03,252] INFO Unable to read additional data from server sessionid 0x13ae8a88aae0005, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) [2012-11-10 04:55:03,353] INFO zookeeper state changed (Disconnected) (org.I0Itec.zkclient.ZkClient) > System Test Transient Failure (case 4011 Log Retention) - ConsoleConsumer > receives less data > -------------------------------------------------------------------------------------------- > > Key: KAFKA-607 > URL: https://issues.apache.org/jira/browse/KAFKA-607 > Project: Kafka > Issue Type: Bug > Reporter: John Fung > Attachments: testcase_4011_data_and_log4j.tar.gz > > -- 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