[ https://issues.apache.org/jira/browse/KAFKA-350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13421235#comment-13421235 ]
Jun Rao commented on KAFKA-350: ------------------------------- Thanks for patch v3. For 21, my point is that the exceptions that can be thrown in partitionAndCollate() are non-recoverable and therefore retries won't help. partitionAndCollate() won't throw NoBrokersForPartitionException since only BrokerPartitionInfo.updateInfo can throw such an exception and updateInfo is not called here. partitionAndCollate() throws InvalidPartitionException that indicates a wrong partition of a topic. If we just retry, we will hit the same exception and fail again. It's simpler to just throw an exception and treat it as a failed case. Ditto for other exceptions that partitionAndCollate() may throw. A few new comments: 30. SyncProducerConfig.requestTimeoutMs: We should make sure that the value is a positive integer and change the comment accordingly. 31. IteraterTemplate: The two KafkaExceptions are better reverted to IllegalStateException. 32. ProducerPerformance: We should remove socketTimeoutMsOpt. 33. SocketServer: no need to import illegalStateException 34. I got the following exception when running system_test/single_host_multi_brokers/bin/run-test.sh 2012-07-24 00:22:51 cleaning up kafka server log/data dir 2012-07-24 00:22:53 starting zookeeper 2012-07-24 00:22:55 starting cluster 2012-07-24 00:22:55 starting kafka server 2012-07-24 00:22:55 -> kafka_pids[1]: 75282 2012-07-24 00:22:55 starting kafka server 2012-07-24 00:22:55 -> kafka_pids[2]: 75286 2012-07-24 00:22:55 starting kafka server 2012-07-24 00:22:55 -> kafka_pids[3]: 75291 2012-07-24 00:22:57 creating topic [mytest] on [localhost:2181] creation failed because of org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /brokers/ids org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /brokers/ids at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47) at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685) at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:413) at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:409) at kafka.utils.ZkUtils$.getChildren(ZkUtils.scala:363) at kafka.utils.ZkUtils$.getSortedBrokerList(ZkUtils.scala:80) at kafka.admin.CreateTopicCommand$.createTopic(CreateTopicCommand.scala:86) at kafka.admin.CreateTopicCommand$.main(CreateTopicCommand.scala:73) at kafka.admin.CreateTopicCommand.main(CreateTopicCommand.scala) Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /brokers/ids at org.apache.zookeeper.KeeperException.create(KeeperException.java:102) at org.apache.zookeeper.KeeperException.create(KeeperException.java:42) at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1249) at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1277) at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:99) at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:416) at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:413) at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675) ... 7 more > Enable message replication in the presence of controlled failures > ----------------------------------------------------------------- > > Key: KAFKA-350 > URL: https://issues.apache.org/jira/browse/KAFKA-350 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.8 > Reporter: Neha Narkhede > Assignee: Neha Narkhede > Attachments: kafka-350-v1.patch, kafka-350-v2.patch, > kafka-350-v3.patch > > > KAFKA-46 introduced message replication feature in the absence of server > failures. This JIRA will improve the log recovery logic and fix other bugs to > enable message replication to happen in the presence of controlled server > failures -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira