[ 
https://issues.apache.org/jira/browse/KAFKA-1738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14198713#comment-14198713
 ] 

schandr commented on KAFKA-1738:
--------------------------------

Here are the additional logs for the same issue

Controller.log
[2014-11-05 10:31:12,441] WARN [Controller-0-to-broker-0-send-thread], 
Controller 0 fails to send a request to broker 
id:0,host:localhost.localdomain,port:9092 (kafka.controller.RequestSendThread)
java.io.EOFException: Received -1 when reading from channel, socket has likely 
been closed.
        at kafka.utils.Utils$.read(Utils.scala:381)
        at 
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54)
        at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
        at 
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
        at kafka.network.BlockingChannel.receive(BlockingChannel.scala:108)
        at 
kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:146)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
[2014-11-05 10:31:12,445] ERROR [Controller-0-to-broker-0-send-thread], 
Controller 0 epoch 7 failed to send request 
Name:UpdateMetadataRequest;Version:0;Controller:0;ControllerEpoch:7;CorrelationId:8;ClientId:id_0-host_null-port_9092;AliveBrokers:id:0,host:localhost.localdomain,port:9092;PartitionState:[topic_30,0]
 -> 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0),[topic_5,0]
 -> 
(LeaderAndIsrInfo:(Leader:-2,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0)
 to broker id:0,host:localhost.localdomain,port:9092. Reconnecting to broker. 
(kafka.controller.RequestSendThread)
java.nio.channels.ClosedChannelException
        at kafka.network.BlockingChannel.send(BlockingChannel.scala:97)
        at 
kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:132)
        at 
kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:131)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
[2014-11-05 10:31:12,448] INFO [Controller-0-to-broker-0-send-thread], 
Controller 0 connected to id:0,host:localhost.localdomain,port:9092 for sending 
state change requests (kafka.controller.RequestSendThread)

Server.log

[2014-11-05 10:31:12,414] DEBUG Got notification sessionid:0x14980b08c110003 
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,415] DEBUG Got WatchedEvent state:SyncConnected 
type:NodeChildrenChanged path:/brokers/topics for sessionid 0x14980b08c110003 
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,415] DEBUG Received event: WatchedEvent 
state:SyncConnected type:NodeChildrenChanged path:/brokers/topics 
(org.I0Itec.zkclient.ZkClient)
[2014-11-05 10:31:12,415] DEBUG New event: ZkEvent[Children of /brokers/topics 
changed sent to 
kafka.controller.PartitionStateMachine$TopicChangeListener@1fc5681] 
(org.I0Itec.zkclient.ZkEventThread)
[2014-11-05 10:31:12,415] DEBUG Leaving process event 
(org.I0Itec.zkclient.ZkClient)
[2014-11-05 10:31:12,415] DEBUG Delivering event #3 ZkEvent[Children of 
/brokers/topics changed sent to 
kafka.controller.PartitionStateMachine$TopicChangeListener@1fc5681] 
(org.I0Itec.zkclient.ZkEventThread)
[2014-11-05 10:31:12,415] DEBUG Got ping response for sessionid: 
0x14980b08c110003 after 0ms (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,416] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 257,3  
replyHeader:: 257,13610,0  request:: '/brokers/topics,T  response:: 
s{6,6,1415130748279,1415130748279,0,23,0,0,0,23,13610}  
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,417] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 258,8  
replyHeader:: 258,13610,0  request:: '/brokers/topics,T  response:: 
v{'topic_23,'topic_18,'topic_22,'topic_17,'topic_25,'topic_16,'topic_24,'topic_15,'topic_14,'topic_13,'topic_12,'topic_11,'topic_19,'topic_5,'topic_7,'topic_6,'Test1,'topic_10,'topic_9,'topic_8,'topic_20,'topic_30,'topic_21}
  (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,420] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 259,4  
replyHeader:: 259,13610,0  request:: '/brokers/topics/topic_30,F  response:: 
#7b2276657273696f6e223a312c22706172746974696f6e73223a7b2230223a5b305d7d7d,s{13610,13610,1415205072414,1415205072414,0,0,0,0,36,0,13610}
  (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,422] DEBUG Replicas assigned to topic [topic_30], 
partition [0] are [List(0)] (kafka.utils.ZkUtils$)
[2014-11-05 10:31:12,422] DEBUG Replicas assigned to topic [topic_30], 
partition [0] are [List(0)] (kafka.utils.ZkUtils$)
[2014-11-05 10:31:12,425] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 260,3  
replyHeader:: 260,13611,0  request:: '/brokers/topics/topic_30,T  response:: 
s{13610,13610,1415205072414,1415205072414,0,0,0,0,36,0,13610}  
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,425] DEBUG Subscribed data changes for 
/brokers/topics/topic_30 (org.I0Itec.zkclient.ZkClient)
[2014-11-05 10:31:12,427] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 261,4  
replyHeader:: 261,13611,0  request:: '/brokers/topics/topic_30,T  response:: 
#7b2276657273696f6e223a312c22706172746974696f6e73223a7b2230223a5b305d7d7d,s{13610,13610,1415205072414,1415205072414,0,0,0,0,36,0,13610}
  (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,430] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 262,4  
replyHeader:: 262,13611,-101  request:: 
'/brokers/topics/topic_30/partitions/0/state,F  response::   
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,434] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 263,1  
replyHeader:: 263,13612,-101  request:: 
'/brokers/topics/topic_30/partitions/0/state,#7b22636f6e74726f6c6c65725f65706f6368223a372c226c6561646572223a302c2276657273696f6e223a312c226c65616465725f65706f6368223a302c22697372223a5b305d7d,v{s{31,s{'world,'anyone}}},0
  response::   (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,435] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 264,1  
replyHeader:: 264,13613,-101  request:: 
'/brokers/topics/topic_30/partitions/0,,v{s{31,s{'world,'anyone}}},0  
response::   (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,437] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 265,1  
replyHeader:: 265,13614,0  request:: 
'/brokers/topics/topic_30/partitions,,v{s{31,s{'world,'anyone}}},0  response:: 
'/brokers/topics/topic_30/partitions  (org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,438] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 266,1  
replyHeader:: 266,13615,0  request:: 
'/brokers/topics/topic_30/partitions/0,,v{s{31,s{'world,'anyone}}},0  
response:: '/brokers/topics/topic_30/partitions/0  
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,439] DEBUG Reading reply sessionid:0x14980b08c110003, 
packet:: clientPath:null serverPath:null finished:false header:: 267,1  
replyHeader:: 267,13616,0  request:: 
'/brokers/topics/topic_30/partitions/0/state,#7b22636f6e74726f6c6c65725f65706f6368223a372c226c6561646572223a302c2276657273696f6e223a312c226c65616465725f65706f6368223a302c22697372223a5b305d7d,v{s{31,s{'world,'anyone}}},0
  response:: '/brokers/topics/topic_30/partitions/0/state  
(org.apache.zookeeper.ClientCnxn)
[2014-11-05 10:31:12,440] TRACE 131 bytes written. 
(kafka.network.BoundedByteBufferSend)
[2014-11-05 10:31:12,440] TRACE 131 bytes written. 
(kafka.network.BoundedByteBufferSend)
[2014-11-05 10:31:12,441] DEBUG Delivering event #3 done 
(org.I0Itec.zkclient.ZkEventThread)
[2014-11-05 10:31:12,448] DEBUG Created socket with SO_TIMEOUT = 90000 
(requested 90000), SO_RCVBUF = 43690 (requested -1), SO_SNDBUF = 84580 
(requested -1). (kafka.network.BlockingChannel)
[2014-11-05 10:31:12,448] DEBUG Created socket with SO_TIMEOUT = 90000 
(requested 90000), SO_RCVBUF = 43690 (requested -1), SO_SNDBUF = 84580 
(requested -1). (kafka.network.BlockingChannel)
[2014-11-05 10:31:12,448] DEBUG Accepted connection from /127.0.0.1 on 
/127.0.0.1:9092. sendBufferSize [actual|requested]: [102400|102400] 
recvBufferSize [actual|requested]: [102400|102400] (kafka.network.Acceptor)
[2014-11-05 10:31:12,448] DEBUG Accepted connection from /127.0.0.1 on 
/127.0.0.1:9092. sendBufferSize [actual|requested]: [102400|102400] 
recvBufferSize [actual|requested]: [102400|102400] (kafka.network.Acceptor)
[2014-11-05 10:31:12,449] TRACE Processor id 1 selection time = 228708013 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,449] TRACE Processor id 1 selection time = 228708013 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,449] DEBUG Processor 1 listening to new connection from 
/127.0.0.1:55693 (kafka.network.Processor)
[2014-11-05 10:31:12,449] DEBUG Processor 1 listening to new connection from 
/127.0.0.1:55693 (kafka.network.Processor)
[2014-11-05 10:31:12,550] TRACE Processor id 7 selection time = 301161349 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,550] TRACE Processor id 7 selection time = 301161349 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,596] TRACE Processor id 3 selection time = 300772884 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,596] TRACE Processor id 3 selection time = 300772884 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,640] TRACE Processor id 6 selection time = 301205873 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,640] TRACE Processor id 6 selection time = 301205873 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,660] TRACE Processor id 8 selection time = 301212710 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,660] TRACE Processor id 8 selection time = 301212710 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,660] TRACE Processor id 9 selection time = 301187091 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,660] TRACE Processor id 9 selection time = 301187091 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,673] TRACE Processor id 0 selection time = 301144174 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,673] TRACE Processor id 0 selection time = 301144174 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,674] TRACE Processor id 4 selection time = 301200540 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,674] TRACE Processor id 4 selection time = 301200540 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,675] TRACE Processor id 2 selection time = 301240208 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,675] TRACE Processor id 2 selection time = 301240208 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,679] TRACE Processor id 5 selection time = 301217390 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,679] TRACE Processor id 5 selection time = 301217390 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE Processor id 1 selection time = 300491401 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE Processor id 1 selection time = 300491401 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE 176 bytes written. 
(kafka.network.BoundedByteBufferSend)
[2014-11-05 10:31:12,750] TRACE 176 bytes written. 
(kafka.network.BoundedByteBufferSend)
[2014-11-05 10:31:12,750] TRACE Processor id 1 selection time = 232897 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE Processor id 1 selection time = 232897 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE 172 bytes read from /127.0.0.1:55693 
(kafka.network.Processor)
[2014-11-05 10:31:12,750] TRACE 172 bytes read from /127.0.0.1:55693 
(kafka.network.Processor)
[2014-11-05 10:31:12,751] TRACE [Kafka Request Handler 2 on Broker 0], Kafka 
request handler 2 on broker 0 handling request 
Request(1,sun.nio.ch.SelectionKeyImpl@3a6a2cf,null,1415205072751,/127.0.0.1:55693)
 (kafka.server.KafkaRequestHandler)
[2014-11-05 10:31:12,751] TRACE [Kafka Request Handler 2 on Broker 0], Kafka 
request handler 2 on broker 0 handling request 
Request(1,sun.nio.ch.SelectionKeyImpl@3a6a2cf,null,1415205072751,/127.0.0.1:55693)
 (kafka.server.KafkaRequestHandler)
[2014-11-05 10:31:12,751] TRACE [KafkaApi-0] Handling request: 
Name:UpdateMetadataRequest;Version:0;Controller:0;ControllerEpoch:7;CorrelationId:8;ClientId:id_0-host_null-port_9092;AliveBrokers:id:0,host:localhost.localdomain,port:9092;PartitionState:[topic_30,0]
 -> 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0),[topic_5,0]
 -> 
(LeaderAndIsrInfo:(Leader:-2,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0)
 from client: /127.0.0.1:55693 (kafka.server.KafkaApis)
[2014-11-05 10:31:12,751] TRACE [KafkaApi-0] Handling request: 
Name:UpdateMetadataRequest;Version:0;Controller:0;ControllerEpoch:7;CorrelationId:8;ClientId:id_0-host_null-port_9092;AliveBrokers:id:0,host:localhost.localdomain,port:9092;PartitionState:[topic_30,0]
 -> 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0),[topic_5,0]
 -> 
(LeaderAndIsrInfo:(Leader:-2,ISR:0,LeaderEpoch:0,ControllerEpoch:7),ReplicationFactor:1),AllReplicas:0)
 from client: /127.0.0.1:55693 (kafka.server.KafkaApis)
[2014-11-05 10:31:12,752] TRACE Processor id 1 selection time = 660576 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,752] TRACE Processor id 1 selection time = 660576 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE Socket server received response to send, 
registering for write: 
Response(1,Request(1,sun.nio.ch.SelectionKeyImpl@3a6a2cf,null,1415205072751,/127.0.0.1:55693),kafka.network.BoundedByteBufferSend@6d0015b6,SendAction)
 (kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE Socket server received response to send, 
registering for write: 
Response(1,Request(1,sun.nio.ch.SelectionKeyImpl@3a6a2cf,null,1415205072751,/127.0.0.1:55693),kafka.network.BoundedByteBufferSend@6d0015b6,SendAction)
 (kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE Processor id 1 selection time = 12017 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE Processor id 1 selection time = 12017 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE 10 bytes written to /127.0.0.1:55693 using key 
sun.nio.ch.SelectionKeyImpl@3a6a2cf (kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE 10 bytes written to /127.0.0.1:55693 using key 
sun.nio.ch.SelectionKeyImpl@3a6a2cf (kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE 6 bytes read. 
(kafka.network.BoundedByteBufferReceive)
[2014-11-05 10:31:12,753] TRACE 6 bytes read. 
(kafka.network.BoundedByteBufferReceive)
[2014-11-05 10:31:12,753] TRACE Finished writing, registering for read on 
connection /127.0.0.1:55693 (kafka.network.Processor)
[2014-11-05 10:31:12,753] TRACE Finished writing, registering for read on 
connection /127.0.0.1:55693 (kafka.network.Processor)
[2014-11-05 10:31:12,851] TRACE Processor id 7 selection time = 301124890 ns 
(kafka.network.Processor)
[2014-11-05 10:31:12,851] TRACE Processor id 7 selection time = 301124890 ns 
(kafka.network.Processor)


If we look at the timestamp, when the error is thrown in the controller.log its 
at 10:31:12,445. But the timestamp in the server.log where the data is written 
to the Socket is after the exception is thrown.

Is there some kind of race condition with the acceptor and send threads within 
the Kafka processor?

> Partitions for topic not created after restart from forced shutdown
> -------------------------------------------------------------------
>
>                 Key: KAFKA-1738
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1738
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.8.1.1, 0.8.2
>         Environment: Linux, 2GB RAM, 2 Core CPU
>            Reporter: Pradeep
>         Attachments: 1738.zip
>
>
> We are using Kafka Topic APIs to create the topic. But in some cases, the 
> topic gets created but we don't see the partition specific files and when 
> producer/consumer tries to get the topic metadata and it fails with 
> exception. Same happens if one tries to create using the command line.
> k.p.BrokerPartitionInfo - Error while fetching metadata [{TopicMetadata for 
> topic tloader1 -> No partition metadata for topic tloader1 due to 
> kafka.common.UnknownTopicOrPartitionException}] for topic [tloader1]: class 
> kafka.common.UnknownTopicOrPartitionException
> Steps to reproduce - 
> 1.      Stop kafka using kill  -9 <PID of Kafka>
> 2.      Start Kafka
> 3.      Create Topic with partition and replication factor of 1.
> 4.      Check the response “Created topic <topic_name>”
> 5.      Run the list command to verify if its created.
> 6.      Now check the data directory of kakfa. There would not be any for the 
> newly created topic.
> We see issues when we are creating new topics. This happens randomly and we 
> dont know the exact reasons. We see the below logs in controller during the 
> time of creation of topics which doesnt have the partition files.
> 2014-11-03 13:12:50,625] INFO [Controller 0]: New topic creation callback for 
> [JobJTopic,0] (kafka.controller.KafkaController)
> [2014-11-03 13:12:50,626] INFO [Controller 0]: New partition creation 
> callback for [JobJTopic,0] (kafka.controller.KafkaController)
> [2014-11-03 13:12:50,626] INFO [Partition state machine on Controller 0]: 
> Invoking state change to NewPartition for partitions [JobJTopic,0] 
> (kafka.controller.PartitionStateMachine)
> [2014-11-03 13:12:50,653] INFO [Replica state machine on controller 0]: 
> Invoking state change to NewReplica for replicas 
> [Topic=JobJTopic,Partition=0,Replica=0] (kafka.controller.ReplicaStateMachine)
> [2014-11-03 13:12:50,654] INFO [Partition state machine on Controller 0]: 
> Invoking state change to OnlinePartition for partitions [JobJTopic,0] 
> (kafka.controller.PartitionStateMachine)
> [2014-11-03 13:12:50,654] DEBUG [Partition state machine on Controller 0]: 
> Live assigned replicas for partition [JobJTopic,0] are: [List(0)] 
> (kafka.controller.PartitionStateMachine)
> [2014-11-03 13:12:50,654] DEBUG [Partition state machine on Controller 0]: 
> Initializing leader and isr for partition [JobJTopic,0] to 
> (Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:2) 
> (kafka.controller.PartitionStateMachine)
> [2014-11-03 13:12:50,667] INFO [Replica state machine on controller 0]: 
> Invoking state change to OnlineReplica for replicas 
> [Topic=JobJTopic,Partition=0,Replica=0] (kafka.controller.ReplicaStateMachine)
> [2014-11-03 13:12:50,794] WARN [Controller-0-to-broker-0-send-thread], 
> Controller 0 fails to send a request to broker id:0,host:DMIPVM,port:9092 
> (kafka.controller.RequestSendThread)
> java.io.EOFException: Received -1 when reading from channel, socket has 
> likely been closed.
>       at kafka.utils.Utils$.read(Utils.scala:381)
>       at 
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54)
>       at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>       at 
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>       at kafka.network.BlockingChannel.receive(BlockingChannel.scala:108)
>       at 
> kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:146)
>       at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> [2014-11-03 13:12:50,965] ERROR [Controller-0-to-broker-0-send-thread], 
> Controller 0 epoch 2 failed to send request 
> Name:UpdateMetadataRequest;Version:0;Controller:0;ControllerEpoch:2;CorrelationId:43;ClientId:id_0-host_null-port_9092;AliveBrokers:id:0,host:DMIPVM,port:9092;PartitionState:[JobJTopic,0]
>  -> 
> (LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:2),ReplicationFactor:1),AllReplicas:0)
>  to broker id:0,host:DMIPVM,port:9092. Reconnecting to broker. 
> (kafka.controller.RequestSendThread)
> java.nio.channels.ClosedChannelException
>       at kafka.network.BlockingChannel.send(BlockingChannel.scala:97)
>       at 
> kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:132)
>       at 
> kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:131)
>       at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to