Joel - that is exactly right. ZkClient has no way to notify consumers of this situation. The session end event gets fired, however, the session begin event never occurs.
Neha - The issue manifested itself when producers were attempting to discover topics/brokers. The kafka brokers had lost their ZK sessions during a network outage. The outage was long enough for ZooKeeper to expire the sessions corresponding to the ephemeral nodes in /broker/. The zkclient bug prevented the broker from ever re-establishing the ZK session. Subsequently, no zookeeper based producer was able to discover topic->broker mappings. The resulting exceptions looked like: Caused by: kafka.common.NoBrokersForPartitionException: Partition = null at kafka.producer.Producer.kafka$producer$Producer$getPartitionListForTopic(Producer.scala:167) at kafka.producer.Producer$anonfun$3.apply(Producer.scala:116) at kafka.producer.Producer$anonfun$3.apply(Producer.scala:105) at scala.collection.TraversableLike$anonfun$map$1.apply(TraversableLike.scala:233) at scala.collection.TraversableLike$anonfun$map$1.apply(TraversableLike.scala:233) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:34) at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:33) at scala.collection.TraversableLike$class.map(TraversableLike.scala:233) at scala.collection.mutable.WrappedArray.map(WrappedArray.scala:33) at kafka.producer.Producer.zkSend(Producer.scala:105) at kafka.producer.Producer.send(Producer.scala:99) at com.yieldmo.common.protobuf.ProtoKafkaWriter$class.write(ProtoKafka.scala:20) at com.yieldmo.common.protobuf.ProtoWriter.write(ProtoKafka.scala:40) at com.yieldmo.storm.bolt.KafkaProtoWriterBolt.execute(KafkaProtoWriterBolt.scala:48) As far as I can see, the only way to deal with this without patching zkclient is to periodically check the status of the zk connection and try to detect this kind of situation. I would love to hear better ideas for how to handle this. On Tue, Sep 24, 2013 at 3:31 AM, Joel Koshy <jjkosh...@gmail.com> wrote: > > node loss. Did the Kafka consumer not respond to rebalance events or did > > the server not respond to state change events ? Also, ephemeral nodes are > > lost only when sessions are expired on the zookeeper server or if clients > > close the session actively, how does losing connection lead to ephemeral > > node loss? > > My understanding of Anatoly's observation is that on session > expiration, zkclient will reconnect > ( > https://github.com/sgroschupf/zkclient/blob/master/src/main/java/org/I0Itec/zkclient/ZkClient.java#L458 > ) > but if the connect causes an IOException, that would effectively mean > that the session will not get re-established. Anatoly, can you > confirm? > > > On Mon, Sep 23, 2013 at 7:02 AM, Anatoly Fayngelerin <fanat...@gmail.com > >wrote: > > > >> Hi Everyone, > >> > >> I've run into the following issue with the Kafka server. The zkclient > lib > >> seems to die silently if there is an UnknownHostException(or any > >> IOException) while reconnecting the ZK session. I've filed a bug about > this > >> with the zkclient lib(https://github.com/sgroschupf/zkclient/issues/23 > ). > >> The > >> ramifications for Kafka were the silent loss of all ephemeral nodes > >> associated with the affected process. > >> > >> Has anyone faced this issue? If so, what is the recommended way of > dealing > >> with this? > >> > >> If there is no good solution available, would the community be open to a > >> patch that periodically verifies ZK connectivity? > >> > >> Thanks, > >> Anatoly > >> >