Uploaded image for project: 'ZooKeeper'
  1. ZooKeeper
  2. ZOOKEEPER-3778

Cannot upgrade from 3.5.7 to 3.6.0 due to multiAddress.reachabilityCheckEnabled

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 3.6.0
    • 3.6.1
    • None
    • None

    Description

      I upgrade our cluster from 3.5.7 to 3.6.0. I make small change in config for metricsProvider (prometheus) which I guess won't affect the our cluster's functions. But we get following error log: 

      2020-04-01 04:04:57,892 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):Follower@292] - shutdown Follower
      2020-04-01 04:04:57,892 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@863] - Peer state changed: looking
      2020-04-01 04:04:57,892 [myid:1] - WARN  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@1501] - PeerState set to LOOKING
      2020-04-01 04:04:57,892 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@1371] - LOOKING
      2020-04-01 04:04:57,892 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):FastLeaderElection@931] - New election. My id = 1, proposed zxid=0x140000044b
      2020-04-01 04:04:57,894 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection$Messenger$WorkerReceiver@376] - Notification: my state:LOOKING; n.sid:1, n.state:LOOKING, n.leader:1, n.round:$
      2020-04-01 04:04:57,895 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection$Messenger$WorkerReceiver@376] - Notification: my state:LOOKING; n.sid:2, n.state:FOLLOWING, n.leader:3, n.roun$
      2020-04-01 04:04:57,896 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection$Messenger$WorkerReceiver@376] - Notification: my state:LOOKING; n.sid:3, n.state:LEADING, n.leader:3, n.round:$
      2020-04-01 04:04:57,896 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@857] - Peer state changed: following
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@1453] - FOLLOWING
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ZooKeeperServer@1246] - minSessionTimeout set to 4000
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ZooKeeperServer@1255] - maxSessionTimeout set to 40000
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ResponseCache@45] - Response cache size is initialized with value 400.
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ResponseCache@45] - Response cache size is initialized with value 400.
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@111] - zookeeper.pathStats.slotCapacity = 60
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@112] - zookeeper.pathStats.slotDuration = 15
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@113] - zookeeper.pathStats.maxDepth = 6
      2020-04-01 04:04:57,897 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@114] - zookeeper.pathStats.initialDelay = 5
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@115] - zookeeper.pathStats.delay = 5
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):RequestPathMetricsCollector@116] - zookeeper.pathStats.enabled = false
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ZooKeeperServer@1470] - The max bytes for all large requests are set to 104857600
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ZooKeeperServer@1484] - The large request threshold is set to -1
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):ZooKeeperServer@329] - Created server with tickTime 2000 minSessionTimeout 4000 maxSes$
      2020-04-01 04:04:57,898 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):Follower@75] - FOLLOWING - LEADER ELECTION TOOK - 5 MS
      2020-04-01 04:04:57,899 [myid:1] - INFO  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):QuorumPeer@863] - Peer state changed: following - discovery
      2020-04-01 04:04:57,900 [myid:1] - WARN  [QuorumPeer[myid=1](plain=[0:0:0:0:0:0:0:0]:2181)(secure=disabled):Follower@129] - Exception when following the leader
      java.lang.IllegalArgumentException
              at java.base/java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1295)
              at java.base/java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1181)
              at java.base/java.util.concurrent.Executors.newFixedThreadPool(Executors.java:92)
              at org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:275)
              at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:87)
              at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1455)
      

       

       After checking the code here

              if (self.isMultiAddressReachabilityCheckEnabled()) {
                  // even if none of the addresses are reachable, we want to try to establish connection
                  // see ZOOKEEPER-3758
                  addresses = multiAddr.getAllReachableAddressesOrAll();
              } else {
                  addresses = multiAddr.getAllAddresses();
              }
      
              ExecutorService executor = Executors.newFixedThreadPool(addresses.size());  
      

      I guess there's something wrong with multiAddress.reachabilityCheckEnabled. So I decide to turn it off (false). After that, I can start our cluster as expected.

      So could you please:

      • Update the document here for multiAddress.reachabilityCheckEnabled because it has effect even if multiAddress.enabled=false (which is default)
      • Check the code in Learner.java to make sure addresses.size() is always larger than 0

      Attachments

        Issue Links

          Activity

            People

              symat Mate Szalay-Beko
              htbn_hoang Hoang Dang
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: