Joe, >> I will modify my kafka server (running off of trunk anyways so no big deal) >> and after if that shows more info then create jira and upload patch I used to see what is going on, np.
Thanks for volunteering to fix it ! I was pointing to line 122 in SimpleConsumer's multifetch() API, not the server side code. -Neha On Wed, Dec 21, 2011 at 12:07 PM, Joe Stein <crypt...@gmail.com> wrote: > Neha, I will investigate #1 further and for #2 I will modify my kafka > server (running off of trunk anyways so no big deal) and after if that > shows more info then create jira and upload patch I used to see what is > going on, np. > > Thanks!!! > > On Wed, Dec 21, 2011 at 2:46 PM, Neha Narkhede <neha.narkh...@gmail.com>wrote: > >> Joe, >> >> I can see 2 problems here - >> >> 1. Zookeeper server/client issue - If you observe the logs, you'll see >> that your Zookeeper cluster is disconnecting your client and expiring >> the session. Typically, this means that the zookeeper server cannot >> receive the client ping, either due to server-side GC or client-side >> GC or something else. I recommend you check the health of your >> zookeeper cluster. See this for more information - >> http://phunt1.wordpress.com/ >> 2. Kafka server issue - When the consumer sent a fetch request to the >> broker, it ran into an IO Exception, but due to a log4j bug ( >> info("multifetch reconnect due to " + e) ), we don't know the cause of >> this IOException. We will need to fix this, to really see why the >> fetch request failed >> >> Thanks, >> Neha >> >> On Wed, Dec 21, 2011 at 8:53 AM, Joe Stein <crypt...@gmail.com> wrote: >> > So I have some new consumers that are not behaving nicely at all. >> > Basically they run for a bit fine and then just stop and die on me. >> > >> > Here are my logs from my consumer. >> > >> > Any assistance would be helpful if you know what I might be doing wrong? >> I >> > am going to dig back through my code and implementation later (with debug >> > on) but something is just not right somewhere. >> > >> > 2011-12-21 10:20:42,068 INFO main >> kafka.consumer.ZookeeperConsumerConnector >> > - end rebalancing consumer test-consumer-group_doozer try #0 >> > 2011-12-21 10:20:42,069 INFO FetchRunnable-0 >> kafka.consumer.FetcherRunnable >> > - FetchRunnable-0 start fetching topic: controserverlogs part: 0 offset: >> > 98266742370 from starscream:9092 >> > Starting consumer thread.. >> > 2011-12-21 10:26:27,393 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:26:28,065 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:26:28,220 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@292e2fba >> > 2011-12-21 10:27:00,713 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:27:00,856 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:27:16,160 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@1e79edfe >> > 2011-12-21 10:27:31,833 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:27:32,667 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@520ed128 >> > 2011-12-21 10:27:32,810 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:28:01,077 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:28:01,948 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:28:01,986 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@246972f1 >> > 2011-12-21 10:28:15,196 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@4bb8d481 >> > 2011-12-21 10:28:29,274 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:28:29,486 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:28:29,925 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@28e73236 >> > 2011-12-21 10:28:56,302 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:28:56,584 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@47122d >> > 2011-12-21 10:28:56,624 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:29:21,337 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Disconnected) >> > 2011-12-21 10:29:21,654 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=3000 >> > watcher=com.medialets.zookeeper.ZooKeeperClient$$anon$1@1ed2e55e >> > 2011-12-21 10:29:22,029 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (Expired) >> > 2011-12-21 10:29:22,029 INFO main-EventThread >> > org.apache.zookeeper.ZooKeeper - Initiating client connection, >> > connectString=zoo1:2181,zoo2:2181,zoo3:2181 sessionTimeout=6000 >> > watcher=org.I0Itec.zkclient.ZkClient@1f758500 >> > 2011-12-21 10:29:22,031 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - ZK expired; release old >> broker >> > parition ownership; re-register consumer pemdas-consumer-group_doozer >> > 2011-12-21 10:29:22,031 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - begin registering consumer >> > pemdas-consumer-group_doozer in ZK >> > 2011-12-21 10:29:22,032 INFO main-EventThread >> org.I0Itec.zkclient.ZkClient >> > - zookeeper state changed (SyncConnected) >> > 2011-12-21 10:29:22,033 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - end registering consumer >> > pemdas-consumer-group_doozer in ZK >> > 2011-12-21 10:29:22,033 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - begin rebalancing consumer >> > pemdas-consumer-group_doozer try #0 >> > 2011-12-21 10:29:22,046 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - Committing all offsets >> > 2011-12-21 10:29:22,046 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - Releasing partition ownership >> > 2011-12-21 10:29:22,046 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - Consumer >> > pemdas-consumer-group_doozer rebalancing the following partitions: >> > List(0-0) for topic controserverlogs with consumers: >> > List(pemdas-consumer-group_doozer-0) >> > 2011-12-21 10:29:22,046 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - >> pemdas-consumer-group_doozer-0 >> > attempting to claim partition 0-0 >> > 2011-12-21 10:29:22,047 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - >> pemdas-consumer-group_doozer-0 >> > successfully owned partition 0-0 >> > 2011-12-21 10:29:22,048 INFO >> > ZkClient-EventThread-121-zoo1:2181,zoo2:2181,zoo3:2181 >> > kafka.consumer.ZookeeperConsumerConnector - Consumer >> > pemdas-consumer-group_doozer selected partitions : controserverlogs:0-0: >> > fetched offset = 99659228357: consumed offset = 99659228357 >> > 2011-12-21 10:29:22,049 INFO FetchRunnable-0 >> kafka.consumer.SimpleConsumer >> > - multifetch reconnect due to >> java.nio.channels.ClosedByInterruptException >> > 2011-12-21 10:29:22,053 INFO FetchRunnable-0 >> kafka.consumer.FetcherRunnable >> > - FecherRunnable Thread[FetchRunnable-0,5,main] interrupted >> > 2011-12-21 10:29:22,053 INFO FetchRunnable-0 >> kafka.consumer.FetcherRunnable >> > - stopping fetcher FetchRunnable-0 to host starscream >> > >> > >> > -- >> > >> > /* >> > Joe Stein >> > http://www.linkedin.com/in/charmalloc >> > Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop> >> > */ >> > > > > -- > > /* > Joe Stein > http://www.linkedin.com/in/charmalloc > Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop> > */