[ https://issues.apache.org/jira/browse/KAFKA-8716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16894026#comment-16894026 ]
Yu Yang commented on KAFKA-8716: -------------------------------- The following is the log (with debug log) around the exception: {code} [2019-07-26 17:45:44,476] INFO Creating /brokers/ids/85 (is it secure? false) (kafka.zk.KafkaZkClient) [2019-07-26 17:45:44,484] DEBUG Reading reply sessionid:0x70000593f202705, packet:: clientPath:null serverPath:null finished:false header:: 91,14 replyHeader:: 91,234840046463,0 request:: org.apache.zookeeper.MultiTransactionRecord@3cd2650b response:: org.apache.zookeeper.MultiResponse@fffff554 (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:44,486] ERROR Error while creating ephemeral at /brokers/ids/85 with return code: SESSIONEXPIRED (kafka.zk.KafkaZkClient$CheckedEphemeral) [2019-07-26 17:45:44,491] ERROR [KafkaServer id=85] Fatal error during KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer) org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired at org.apache.zookeeper.KeeperException.create(KeeperException.java:134) at kafka.zk.KafkaZkClient$CheckedEphemeral.create(KafkaZkClient.scala:1727) {code} The following is debug log from ZooKeeperClientWatcher: {code} [2019-07-26 17:45:43,296] DEBUG [ZooKeeperClient Kafka server] Received event: WatchedEvent state:SyncConnected type:None path:null (kafka.zookeeper.ZooKeeperClient) [2019-07-26 17:45:43,449] DEBUG [ZooKeeperClient Kafka server] Received event: WatchedEvent state:Closed type:None path:null (kafka.zookeeper.ZooKeeperClient) [2019-07-26 17:45:43,489] DEBUG [ZooKeeperClient Kafka server] Received event: WatchedEvent state:SyncConnected type:None path:null (kafka.zookeeper.ZooKeeperClient) [2019-07-26 17:45:44,901] DEBUG [ZooKeeperClient Kafka server] Received event: WatchedEvent state:Closed type:None path:null (kafka.zookeeper.ZooKeeperClient) {code} The following is the log for the zookeeper session: {code} [2019-07-26 17:45:43,489] INFO Session establishment complete on server datazk007/10.1.16.191:2181, sessionid = 0x70000593f202705, negotiated timeout = 6000 (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:43,492] DEBUG Reading reply sessionid:0x70000593f202705, packet:: clientPath:/consumers serverPath:/testkafka/consumers finished:false header:: 1,1 replyHeader:: 1,234840045921,-110 request:: '/testkafka/consumers,,v{s{31,s{'world,'anyone}}},0 response:: (org.apache.zookeeper.ClientCnxn) ... [2019-07-26 17:45:44,484] DEBUG Reading reply sessionid:0x70000593f202705, packet:: clientPath:null serverPath:null finished:false header:: 91,14 replyHeader:: 91,234840046463,0 request:: org.apache.zookeeper.MultiTransactionRecord@3cd2650b response:: org.apache.zookeeper.MultiResponse@fffff554 (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:44,800] DEBUG Closing session: 0x70000593f202705 (org.apache.zookeeper.ZooKeeper) [2019-07-26 17:45:44,800] DEBUG Closing client for session: 0x70000593f202705 (org.apache.zookeeper.ClientCnxn) ... [2019-07-26 17:45:44,800] DEBUG Reading reply sessionid:0x70000593f202705, packet:: clientPath:null serverPath:null finished:false header:: 92,-11 replyHeader:: 92,234840046569,0 request:: null response:: null (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:44,800] DEBUG Disconnecting client for session: 0x70000593f202705 (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:44,800] DEBUG An exception was thrown while closing send thread for session 0x70000593f202705 : Unable to read additional data from server sessionid 0x70000593f202705, likely server has closed socket (org.apache.zookeeper.ClientCnxn) [2019-07-26 17:45:44,901] INFO Session: 0x70000593f202705 closed (org.apache.zookeeper.ZooKeeper) [2019-07-26 17:45:44,901] INFO EventThread shut down for session: 0x70000593f202705 (org.apache.zookeeper.ClientCnxn) {code} > broker cannot join the cluster after upgrading kafka binary from 2.1.1 to > 2.2.1 or 2.3.0 > ---------------------------------------------------------------------------------------- > > Key: KAFKA-8716 > URL: https://issues.apache.org/jira/browse/KAFKA-8716 > Project: Kafka > Issue Type: Bug > Components: zkclient > Affects Versions: 2.3.0, 2.2.1 > Reporter: Yu Yang > Priority: Critical > > We are trying to upgrade kafka binary from 2.1 to 2.2.1 or 2.3.0. For both > versions, the broker with updated binary (2.2.1 or 2.3.0) could not get > started due to zookeeper session expiration exception. This error happens > repeatedly and the broker could not start because of this. > Below is our zk related setting in server.properties: > {code} > zookeeper.connection.timeout.ms=6000 > zookeeper.session.timeout.ms=6000 > {code} > The following is the stack trace, and we are using zookeeper 3.5.3. Instead > of waiting for a few seconds, the SESSIONEXPIRED error returned immediately > in CheckedEphemeral.create call. Any insights? > [2019-07-25 18:07:35,712] INFO Creating /brokers/ids/80 (is it secure? false) > (kafka.zk.KafkaZkClient) > [2019-07-25 18:07:35,724] ERROR Error while creating ephemeral at > /brokers/ids/80 with return code: SESSIONEXPIRED > (kafka.zk.KafkaZkClient$CheckedEphemeral) > [2019-07-25 18:07:35,731] ERROR [KafkaServer id=80] Fatal error during > KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer) > org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode > = Session expired > at org.apache.zookeeper.KeeperException.create(KeeperException.java:130) > at kafka.zk.KafkaZkClient$CheckedEphemeral.create(KafkaZkClient.scala:1725) > at kafka.zk.KafkaZkClient.checkedEphemeralCreate(KafkaZkClient.scala:1689) > at kafka.zk.KafkaZkClient.registerBroker(KafkaZkClient.scala:97) > at kafka.server.KafkaServer.startup(KafkaServer.scala:260) > at kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:38) > at kafka.Kafka$.main(Kafka.scala:75) > at kafka.Kafka.main(Kafka.scala) -- This message was sent by Atlassian JIRA (v7.6.14#76016)