Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-22601

Misconfigured addition of peers leads to cluster shutdown.

    XMLWordPrintableJSON

Details

    • Reviewed

    Description

      Recently we added a peer to a production cluster which were in different kerberos realm.

      Steps to reproduce:
      1. Add a misconfigured peer which is in different kerberos realm.
      2. Remove that peer.
      3. All region servers will start to crash.

      RCA
      Enabled trace logging on one Region server for a short amount of time.
      After adding peer, saw the following log lines.

      2019-06-18 22:19:20,949 INFO  [main-EventThread] replication.ReplicationTrackerZKImpl - /hbase/replication/peers znode expired, triggering peerListChanged event
      2019-06-18 22:19:20,992 INFO  [main-EventThread] replication.ReplicationPeersZKImpl - Added new peer cluster=<dev-cluster-zookeeper>:/hbase
      2019-06-18 22:19:21,113 INFO  [main-EventThread] zookeeper.RecoverableZooKeeper - Process identifier=hconnection-0x794a56d6 connecting to ZooKeeper ensemble=<dev-cluster-zookeeper>
      
      2019-06-18 22:20:01,280 WARN  [main-EventThread] zookeeper.ZKUtil - hconnection-0x794a56d6-0x16b56265fbebb1b, quorum=<dev-cluster-zookeeper>, baseZNode=/hbase Unable to set watcher on znode (/hbase/hbaseid)
      org.apache.zookeeper.KeeperException$AuthFailedException: KeeperErrorCode = AuthFailed for /hbase/hbaseid
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:123)
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
              at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1102)
              at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:220)
              at org.apache.hadoop.hbase.zookeeper.ZKUtil.checkExists(ZKUtil.java:421)
              at org.apache.hadoop.hbase.zookeeper.ZKClusterId.readClusterIdZNode(ZKClusterId.java:65)
              at org.apache.hadoop.hbase.client.ZooKeeperRegistry.getClusterId(ZooKeeperRegistry.java:105)
              at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.retrieveClusterId(ConnectionManager.java:922)
              at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.<init>(ConnectionManager.java:706)
              at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.<init>(ConnectionManager.java:638)
              at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
              at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
              at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
              at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
              at org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:238)
              at org.apache.hadoop.hbase.client.ConnectionManager.createConnection(ConnectionManager.java:432)
              at org.apache.hadoop.hbase.client.ConnectionManager.createConnectionInternal(ConnectionManager.java:341)
              at org.apache.hadoop.hbase.client.HConnectionManager.createConnection(HConnectionManager.java:144)
              at org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint.init(HBaseInterClusterReplicationEndpoint.java:135)
              at com.salesforce.hbase.replication.TenantReplicationEndpoint.init(TenantReplicationEndpoint.java:30)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.getReplicationSource(ReplicationSourceManager.java:517)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.addSource(ReplicationSourceManager.java:273)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.peerListChanged(ReplicationSourceManager.java:635)
              at org.apache.hadoop.hbase.replication.ReplicationTrackerZKImpl$PeersWatcher.nodeChildrenChanged(ReplicationTrackerZKImpl.java:192)
              at org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:643)
              at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:544)
              at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:519)
      
      
      2019-06-18 22:20:42,999 WARN  [Source,<dev-cluster>] zookeeper.ZKUtil - connection to cluster: <dev-cluster>-0x26b56265fe7b5cd, quorum=<dev-cluster-zookeeper>, baseZNode=/hbase Unable to set watcher on znode (/hbase/hbaseid)
      
      org.apache.zookeeper.KeeperException$AuthFailedException: KeeperErrorCode = AuthFailed for /hbase/hbaseid
      
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:123)
      
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
      
              at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1102)
      
              at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:220)
      
              at org.apache.hadoop.hbase.zookeeper.ZKUtil.checkExists(ZKUtil.java:421)
      
              at org.apache.hadoop.hbase.zookeeper.ZKClusterId.readClusterIdZNode(ZKClusterId.java:65)
      
              at org.apache.hadoop.hbase.zookeeper.ZKClusterId.getUUIDForCluster(ZKClusterId.java:96)
      
              at org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint.getPeerUUID(HBaseReplicationEndpoint.java:104)
      
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.run(ReplicationSource.java:304)
      
      
      2019-06-18 22:20:43,002 TRACE [Source,<dev-cluster>] regionserver.ReplicationSource - Cannot contact the peer's zk ensemble, sleeping 1000 times 1
      <same stack trace as before>
      
      2019-06-18 22:20:44,008 TRACE [Source,<dev-cluster>] regionserver.ReplicationSource - Cannot contact the peer's zk ensemble, sleeping 1000 times 2
      <same stack trace as before>
      

      This goes on and on until we removed the peer.
      After removing the peer,

      2019-06-18 22:21:20,731 INFO  [main-EventThread] replication.ReplicationTrackerZKImpl - /hbase/replication/peers/<dev-cluster> znode expired, triggering peerRemoved event
      2019-06-18 22:21:20,731 INFO  [main-EventThread] regionserver.ReplicationSourceManager - Closing the following queue <dev-cluster>, currently have 2 and another 0 that were recovered
      2019-06-18 22:21:20,733 INFO  [main-EventThread] regionserver.ReplicationSourceManager - Number of deleted recovered sources for <dev-cluster>: 0
      2019-06-18 22:21:20,734 INFO  [main-EventThread] regionserver.ReplicationSource - Closing source <dev-cluster> because: Replication stream was removed by a user
      2019-06-18 22:21:20,734 INFO  [main-EventThread] replication.TenantReplicationEndpoint - Stopping endpoint
      2019-06-18 22:21:20,736 INFO  [main-EventThread] client.ConnectionManager$HConnectionImplementation - Closing zookeeper sessionid=0x16b56265fbebb1b
      2019-06-18 22:21:20,736 DEBUG [main-EventThread] ipc.RpcClientImpl - Stopping rpc client
      2019-06-18 22:21:20,738 INFO  [main-EventThread] regionserver.ReplicationSourceManager - Done with the queue <dev-cluster>
      2019-06-18 22:21:20,744 DEBUG [main-EventThread] replication.ReplicationQueuesZKImpl - Peer /hbase/replication/hfile-refs/<dev-cluster> not found in hfile reference queue.
      
      
      2019-06-18 22:21:28,066 INFO  [Source,<dev-cluster>] regionserver.ReplicationSource - Replicating d12a5eb3-16bd-4910-91ed-41e0e7990007 -> null
      2019-06-18 22:21:28,067 DEBUG [Source,<dev-cluster>] regionserver.ReplicationSource - Starting up worker for wal group <rs-serner-name>%2C60020%2C1560896050398
      
      2019-06-18 22:21:28,069 ERROR [050398,<dev-cluster>] regionserver.ReplicationSource - Unexpected exception in ReplicationSourceWorkerThread, currentPath=null
      java.lang.IllegalArgumentException: Peer with id= <dev-cluster> is not connected
              at org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl.getStatusOfPeer(ReplicationPeersZKImpl.java:217)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.isPeerEnabled(ReplicationSource.java:363)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceWorkerThread.run(ReplicationSource.java:549)
      
      2019-06-18 22:21:28,070 INFO  [050398,<dev-cluster>] regionserver.HRegionServer - STOPPED: Unexpected exception in ReplicationSourceWorkerThread
      2019-06-18 22:21:28,071 INFO  [/10.231.90.212:60020] regionserver.SplitLogWorker - Sending interrupt to stop the worker thread
      2019-06-18 22:21:28,073 INFO  [/10.231.90.212:60020] regionserver.HRegionServer - Stopping infoServer
      2019-06-18 22:21:28,075 INFO  [as-dnds4-3-prd:60020] regionserver.SplitLogWorker - SplitLogWorker interrupted. Exiting. 
      2019-06-18 22:21:28,076 INFO  [as-dnds4-3-prd:60020] regionserver.SplitLogWorker - SplitLogWorker <rs-server-name>,60020,1560896050398 exiting
      2019-06-18 22:21:28,181 INFO  [/10.231.90.212:60020] regionserver.HeapMemoryManager - Stopping HeapMemoryTuner chore.
      2019-06-18 22:21:28,181 INFO  [/10.231.90.212:60020] flush.RegionServerFlushTableProcedureManager - Stopping region server flush procedure manager gracefully.
      

      Also verified from jstacks I captured between addition of peer and removal of peer, the Replication source thread was sleeping waiting to connect to peer.

      "main-EventThread.replicationSource,<dev-cluster>" #1923 daemon prio=5 os_prio=0 tid=0x00007f1ad18f4800 nid=0x25a47 
      waiting on condition [0x00007f1220314000]
         java.lang.Thread.State: TIMED_WAITING (sleeping)
              at java.lang.Thread.sleep(Native Method)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.sleepForRetries(ReplicationSource.java:349)
              at org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.run(ReplicationSource.java:306)
      

      According to me, the bug lies here:

      ReplicationSource.java
      // Some comments here
        public void run() {
      {
      ....
      ....
       while (this.isSourceActive() && this.peerClusterId == null) {
            this.peerClusterId = replicationEndpoint.getPeerUUID();
            if (this.isSourceActive() && this.peerClusterId == null) {
              if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
                sleepMultiplier++;
              }
            }
          }
      
      !-- There is a bug here. We should check that peerClusterId is not null. If null, then terminate the thread. -->
      
          // In rare case, zookeeper setting may be messed up. That leads to the incorrect
          // peerClusterId value, which is the same as the source clusterId
          if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
            this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
                + peerClusterId + " which is not allowed by ReplicationEndpoint:"
                + replicationEndpoint.getClass().getName(), null, false);
            this.manager.closeQueue(this);
            return;
          }
      ....
      ....
      }
      

      Attachments

        Activity

          People

            shahrs87 Rushabh Shah
            shahrs87 Rushabh Shah
            Votes:
            0 Vote for this issue
            Watchers:
            9 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: