[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Jun Rao (JIRA)


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

Jun Rao commented on KAFKA-8716:


Currently, we don't handle ZK session expiration for the initial broker 
registration in ZK when the broker is started since it's rare. Does that happen 
repeatedly on restart?

> 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
>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.  
> The following is the stack trace, and we are using zookeeper 3.5.3. 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Yu Yang (JIRA)


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

Yu Yang commented on KAFKA-8716:


Thank for checking [~junrao].  Added more information in the description 
session. 

The  "SESSIONExpiration" exception happened immediately after the 
"CheckedEphemeral.create" call, and happened repeatedly so that the broker 
could not get started. 

> 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
>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.  
> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Jun Rao (JIRA)


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

Jun Rao commented on KAFKA-8716:


Hmm, that's kind of weird. We do wait for the ZK session to be connected in the 
constructor of ZooKeeperClient. 

{code:java}
  try waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS)
  catch {
case e: Throwable =>
  close()
  throw e
  }
{code}


It should be rare for the session to be expired immediately after the 
connection is established. Could you turn on the debug level logging in 
ZooKeeperClientWatcher so that we can see logs like the following?
{code:java}
debug(s"Received event: $event"){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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Yu Yang (JIRA)


[ 
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:0x7593f202705, 
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@f554 (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 = 0x7593f202705, negotiated timeout = 
6000 (org.apache.zookeeper.ClientCnxn)
[2019-07-26 17:45:43,492] DEBUG Reading reply sessionid:0x7593f202705, 
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:0x7593f202705, 
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@f554 (org.apache.zookeeper.ClientCnxn)
[2019-07-26 17:45:44,800] DEBUG Closing session: 0x7593f202705 
(org.apache.zookeeper.ZooKeeper)
[2019-07-26 17:45:44,800] DEBUG Closing client for session: 0x7593f202705 
(org.apache.zookeeper.ClientCnxn)
...
[2019-07-26 17:45:44,800] DEBUG Reading reply sessionid:0x7593f202705, 
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: 
0x7593f202705 (org.apache.zookeeper.ClientCnxn)
[2019-07-26 17:45:44,800] DEBUG An exception was thrown while closing send 
thread for session 0x7593f202705 : Unable to read additional data from 
server sessionid 0x7593f202705, likely server has closed socket 
(org.apache.zookeeper.ClientCnxn)
[2019-07-26 17:45:44,901] INFO Session: 0x7593f202705 closed 
(org.apache.zookeeper.ZooKeeper)
[2019-07-26 17:45:44,901] INFO EventThread shut down for session: 
0x7593f202705 (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 

[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Jun Rao (JIRA)


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

Jun Rao commented on KAFKA-8716:


It's weird that the session was closed very quickly after SyncConnected. Is 
there any log on ZK server that indicates why the session was closed?

> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-26 Thread Yu Yang (JIRA)


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

Yu Yang commented on KAFKA-8716:


[~junrao]  it seems that it is related to the zookeeper version that we uses. 
we are using zookeeper 3.5.3 that is a beta version. will upgrade zookeeper to 
3.5.5 that is stable release to see if that fixes the issue. 

> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-30 Thread Xu Zhiyuan (JIRA)


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

Xu Zhiyuan commented on KAFKA-8716:
---

[~yuyang08] Although kafka integrate high version of zookeeper in the binary, 
but kafka document says that the current stable branch of zookeeper is 3.4 and 
the latest release of that branch is 3.4.9. Maybe 3.4.9 is a choice.

[zookeeper|[https://kafka.apache.org/23/documentation.html#zk]]

> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-07-30 Thread Ismael Juma (JIRA)


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

Ismael Juma commented on KAFKA-8716:


The documentation is out of date. I'll submit a PR to update it.

> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-08-07 Thread Yu Yang (JIRA)


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

Yu Yang commented on KAFKA-8716:


Update: We verified that after upgrading zookeeper to 3.5.5,  nodes with kafka 
2.3 binary can re-join the cluster fine.  Thanks for looking into this issue! 

> 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)


[jira] [Commented] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-08-07 Thread Jun Rao (JIRA)


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

Jun Rao commented on KAFKA-8716:


[~yuyang08], thanks for the update.

> 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)