[
https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15187284#comment-15187284
]
Simon Cooper commented on KAFKA-3296:
-------------------------------------
Are the logs useful?
We've also started seeing a (possibly unrelated) issue - a new kafka install
doesn't work, putting this in the logs:
{code}2016-03-09T15:43:22,998 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:delete cxid:0x93 zxid:0x579
txntype:-1 reqpath:n/a Error Path:/kafkasite1/admin/preferred_replica_election
Error:KeeperErrorCode = NoNode for /kafkasite1/admin/preferred_replica_election
2016-03-09T15:43:23,142 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0x9a zxid:0x57a
txntype:-1 reqpath:n/a Error Path:/kafkasite1/brokers Error:KeeperErrorCode =
NodeExists for /kafkasite1/brokers
2016-03-09T15:43:23,142 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0x9b zxid:0x57b
txntype:-1 reqpath:n/a Error Path:/kafkasite1/brokers/ids Error:KeeperErrorCode
= NodeExists for /kafkasite1/brokers/ids
2016-03-09T15:43:23,309 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xa2 zxid:0x57d
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/19
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/19
2016-03-09T15:43:23,313 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xa3 zxid:0x57e
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions
2016-03-09T15:43:23,325 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xa7 zxid:0x582
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/30
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/30
2016-03-09T15:43:23,341 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xaa zxid:0x585
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/47
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/47
2016-03-09T15:43:23,349 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xad zxid:0x588
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/29
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/29
2016-03-09T15:43:23,357 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xb1 zxid:0x58b
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/41
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/41
2016-03-09T15:43:23,367 | INFO | o.a.z.s.PrepRequestProcessor
[ProcessThread(sid:0 cport:-1):] | Got user-level KeeperException when
processing sessionid:0x1535bf80f4e004a type:create cxid:0xba zxid:0x58e
txntype:-1 reqpath:n/a Error
Path:/kafkasite1/brokers/topics/__consumer_offsets/partitions/39
Error:KeeperErrorCode = NoNode for
/kafkasite1/brokers/topics/__consumer_offsets/partitions/39{code}
Any client connection can't get topic metadata until both zookeeper and kafka
are restarted
> All consumer reads hang indefinately
> ------------------------------------
>
> Key: KAFKA-3296
> URL: https://issues.apache.org/jira/browse/KAFKA-3296
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.9.0.0, 0.9.0.1
> Reporter: Simon Cooper
> Priority: Critical
> Attachments: controller.zip, kafkalogs.zip
>
>
> We've got several integration tests that bring up systems on VMs for testing.
> We've recently upgraded to 0.9, and very occasionally we occasionally see an
> issue where every consumer that tries to read from the broker hangs, spamming
> the following in their logs:
> {code}2016-02-26T12:25:37,856 | DEBUG | o.a.k.c.NetworkClient
> [pool-10-thread-1] | Sending metadata request
> ClientRequest(expectResponse=true, callback=null,
> request=RequestSend(header={api_key=3,api_version=0,correlation_id=21905,client_id=consumer-1},
> body={topics=[Topic1]}), isInitiatedByNetworkClient,
> createdTimeMs=1456489537856, sendTimeMs=0) to node 1
> 2016-02-26T12:25:37,856 | DEBUG | o.a.k.c.Metadata [pool-10-thread-1] |
> Updated cluster metadata version 10954 to Cluster(nodes = [Node(1,
> server.internal, 9092)], partitions = [Partition(topic = Topic1, partition =
> 0, leader = 1, replicas = [1,], isr = [1,]])
> 2016-02-26T12:25:37,856 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Issuing group metadata request to broker 1
> 2016-02-26T12:25:37,857 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Group metadata response
> ClientResponse(receivedTimeMs=1456489537857, disconnected=false,
> request=ClientRequest(expectResponse=true,
> callback=org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler@28edb273,
>
> request=RequestSend(header={api_key=10,api_version=0,correlation_id=21906,client_id=consumer-1},
> body={group_id=}), createdTimeMs=1456489537856, sendTimeMs=1456489537856),
> responseBody={error_code=15,coordinator={node_id=-1,host=,port=-1}})
> 2016-02-26T12:25:37,956 | DEBUG | o.a.k.c.NetworkClient [pool-10-thread-1] |
> Sending metadata request ClientRequest(expectResponse=true, callback=null,
> request=RequestSend(header={api_key=3,api_version=0,correlation_id=21907,client_id=consumer-1},
> body={topics=[Topic1]}), isInitiatedByNetworkClient,
> createdTimeMs=1456489537956, sendTimeMs=0) to node 1
> 2016-02-26T12:25:37,956 | DEBUG | o.a.k.c.Metadata [pool-10-thread-1] |
> Updated cluster metadata version 10955 to Cluster(nodes = [Node(1,
> server.internal, 9092)], partitions = [Partition(topic = Topic1, partition =
> 0, leader = 1, replicas = [1,], isr = [1,]])
> 2016-02-26T12:25:37,956 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Issuing group metadata request to broker 1
> 2016-02-26T12:25:37,957 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Group metadata response
> ClientResponse(receivedTimeMs=1456489537957, disconnected=false,
> request=ClientRequest(expectResponse=true,
> callback=org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler@40cee8cc,
>
> request=RequestSend(header={api_key=10,api_version=0,correlation_id=21908,client_id=consumer-1},
> body={group_id=}), createdTimeMs=1456489537956, sendTimeMs=1456489537956),
> responseBody={error_code=15,coordinator={node_id=-1,host=,port=-1}})
> 2016-02-26T12:25:38,056 | DEBUG | o.a.k.c.NetworkClient [pool-10-thread-1] |
> Sending metadata request ClientRequest(expectResponse=true, callback=null,
> request=RequestSend(header={api_key=3,api_version=0,correlation_id=21909,client_id=consumer-1},
> body={topics=[Topic1]}), isInitiatedByNetworkClient,
> createdTimeMs=1456489538056, sendTimeMs=0) to node 1
> 2016-02-26T12:25:38,056 | DEBUG | o.a.k.c.Metadata [pool-10-thread-1] |
> Updated cluster metadata version 10956 to Cluster(nodes = [Node(1,
> server.internal, 9092)], partitions = [Partition(topic = Topic1, partition =
> 0, leader = 1, replicas = [1,], isr = [1,]])
> 2016-02-26T12:25:38,056 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Issuing group metadata request to broker 1
> 2016-02-26T12:25:38,057 | DEBUG | o.a.k.c.c.i.AbstractCoordinator
> [pool-10-thread-1] | Group metadata response
> ClientResponse(receivedTimeMs=1456489538057, disconnected=false,
> request=ClientRequest(expectResponse=true,
> callback=org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler@439e25fb,
>
> request=RequestSend(header={api_key=10,api_version=0,correlation_id=21910,client_id=consumer-1},
> body={group_id=}), createdTimeMs=1456489538056, sendTimeMs=1456489538056),
> responseBody={error_code=15,coordinator={node_id=-1,host=,port=-1}}){code}
> This persists for any 0.9 consumer trying to read from the topic (we haven't
> confirmed if this is for a single topic or for any topic on the broker). 0.8
> consumers can read from the broker without issues. This is fixed by a broker
> restart.
> This was observed on a single-broker cluster. There were no suspicious log
> messages on the server.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)