You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/08/08 11:07:37 UTC

hbase git commit: HBASE-16367 Race between master and region server initialization may lead to premature server abort

Repository: hbase
Updated Branches:
  refs/heads/master e5f9df1e2 -> 50f3c9572


HBASE-16367 Race between master and region server initialization may lead to premature server abort


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/50f3c957
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/50f3c957
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/50f3c957

Branch: refs/heads/master
Commit: 50f3c9572c7c9dcf276b19c137ad600f368743ec
Parents: e5f9df1
Author: tedyu <yu...@gmail.com>
Authored: Mon Aug 8 04:07:30 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 8 04:07:30 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/HMaster.java    |  3 +++
 .../apache/hadoop/hbase/regionserver/HRegionServer.java | 12 ++++++++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/50f3c957/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 4e6952a..5ce056d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -40,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -451,6 +452,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     // Some unit tests don't need a cluster, so no zookeeper at all
     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
+      setInitLatch(new CountDownLatch(1));
       activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
       int infoPort = putUpJettyServer();
       startActiveMasterManager(infoPort);
@@ -693,6 +695,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // publish cluster ID
     status.setStatus("Publishing Cluster ID in ZooKeeper");
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
+    this.initLatch.countDown();
 
     this.serverManager = createServerManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/50f3c957/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e03993f..ba5d669 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -55,6 +55,8 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -233,6 +235,7 @@ public class HRegionServer extends HasThread implements
   protected MemStoreFlusher cacheFlusher;
 
   protected HeapMemoryManager hMemManager;
+  protected CountDownLatch initLatch = null;
 
   /**
    * Cluster connection to be shared by services.
@@ -655,6 +658,10 @@ public class HRegionServer extends HasThread implements
       this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
   }
 
+  protected void setInitLatch(CountDownLatch latch) {
+    this.initLatch = latch;
+  }
+
   /*
    * Returns true if configured hostname should be used
    */
@@ -799,6 +806,8 @@ public class HRegionServer extends HasThread implements
    * @throws IOException
    * @throws InterruptedException
    */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE",
+    justification="cluster Id znode read would give us correct response")
   private void initializeZooKeeper() throws IOException, InterruptedException {
     // Create the master address tracker, register with zk, and start it.  Then
     // block until a master is available.  No point in starting up if no master
@@ -809,6 +818,9 @@ public class HRegionServer extends HasThread implements
     // when ready.
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
+    if (this.initLatch != null) {
+      this.initLatch.await(50, TimeUnit.SECONDS);
+    }
     // Retrieve clusterId
     // Since cluster status is now up
     // ID should have already been set by HMaster