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

Replication sink list is not updated on UnknownHostException

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.2.0, 2.0.0
    • 1.4.1, 2.0.0-beta-2, 1.3.3, 2.0.0
    • Replication
    • None
    • We have two clusters set up with bi-directional replication. The clusters are around 400 nodes each and hosted in AWS.

    • Reviewed

    Description

      We have two clusters, call them 1 and 2. Cluster 1 was the current "primary" cluster and taking all live traffic which is replicated to cluster 2. We decommissioned several instances in cluster 2 which involves deleting the instance and its DNS record. After this happened most of the regions servers in cluster 1 showed this message in their logs repeatedly. 
       

      2018-01-12 23:49:36,507 WARN org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint: Can't replicate because of a local or network error:
      java.net.UnknownHostException: data-017b.hbase-2.prod
      at org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.<init>(AbstractRpcClient.java:315)
      at org.apache.hadoop.hbase.ipc.AbstractRpcClient.createBlockingRpcChannel(AbstractRpcClient.java:267)
      at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.getAdmin(ConnectionManager.java:1737)
      at org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.getAdmin(ConnectionManager.java:1719)
      at org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.getReplicationSink(ReplicationSinkManager.java:119)
      at org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint$Replicator.call(HBaseInterClusterReplicationEndpoint.java:339)
      at org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint$Replicator.call(HBaseInterClusterReplicationEndpoint.java:326)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      at java.lang.Thread.run(Thread.java:748)
      

      The host data-017b.hbase-2.prod was one of those that had been removed from cluster 2. Next we observed our replication lag from cluster 1 to cluster 2 was elevated. Some region servers reported ageOfLastShippedOperation to be close to an hour.

      The only way we found to clear the message was to restart the region servers that showed this message in the log. Once we did replication returned to normal. Restarting the affected region servers in cluster 1 took several days because we could not bring the cluster down.

      From reading the code it appears the cause was the zookeeper watch not being triggered for the region server list change in cluster 2. We verified the list in zookeeper for cluster 2 was correct and did not include the removed nodes.

      One concrete improvement to make would be to force a refresh of the sink cluster region server list when an UnknownHostException is found. This is already done if the there is a ConnectException in HBaseInterClusterReplicationEndpoint.java

      } else if (ioe instanceof ConnectException) {
        LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
        replicationSinkMgr.chooseSinks();
      

      I propose that should be extended to cover UnknownHostException.

      We observed this behavior on 1.2.0-cdh-5.11.1 but it appears the same code still exists on the current master branch.

       

      Attachments

        1. HBASE-19816.master.002.patch
          2 kB
          Scott Wilson

        Activity

          People

            srwilson Scott Wilson
            srwilson Scott Wilson
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: