You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/08/26 17:15:19 UTC

[01/12] HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

Repository: hbase
Updated Branches:
  refs/heads/0.98 5aa39bb3f -> d48752688


http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
new file mode 100644
index 0000000..d9e8480
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
@@ -0,0 +1,45 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.LargeTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Like {@link TestRegionMergeTransaction} in that we're testing
+ * {@link RegionMergeTransaction} only the below tests are against a running
+ * cluster where {@link TestRegionMergeTransaction} is tests against bare
+ * {@link HRegion}.
+ */
+@Category(LargeTests.class)
+public class TestZKLessMergeOnCluster extends TestRegionMergeTransactionOnCluster {
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    // Don't use ZK for region assignment
+    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
+    setupOnce();
+  }
+
+  @AfterClass
+  public static void afterAllTests() throws Exception {
+    TestRegionMergeTransactionOnCluster.afterAllTests();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
new file mode 100644
index 0000000..1e3de64
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
@@ -0,0 +1,45 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.LargeTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
+ * only the below tests are against a running cluster where {@link TestSplitTransaction}
+ * is tests against a bare {@link HRegion}.
+ */
+@Category(LargeTests.class)
+public class TestZKLessSplitOnCluster extends TestSplitTransactionOnCluster {
+  @BeforeClass
+  public static void before() throws Exception {
+    // Don't use ZK for region assignment
+    TESTING_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
+    setupOnce();
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    TestSplitTransactionOnCluster.after();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
index f864ec5..ac77b08 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
@@ -62,6 +62,7 @@ public class TestCloseRegionHandler {
   private int testIndex = 0;
 
   @BeforeClass public static void before() throws Exception {
+    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     HTU.startMiniZKCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
index 7c7cf5a..9df436e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
@@ -54,6 +54,7 @@ public class TestOpenRegionHandler {
   private int testIndex = 0;
 
   @BeforeClass public static void before() throws Exception {
+    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     HTU.startMiniZKCluster();
     TEST_HTD = new HTableDescriptor(TableName.valueOf("TestOpenRegionHandler.java"));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index b815c54..62149e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -1602,6 +1602,12 @@ public class TestHBaseFsck {
       assertFalse(fsck.shouldRerun());
       assertErrors(fsck, new ERROR_CODE[] {
         ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+
+      // fix the cluster so other tests won't be impacted
+      fsck = doFsck(conf, true);
+      assertTrue(fsck.shouldRerun());
+      fsck = doFsck(conf, true);
+      assertNoErrors(fsck);
     } finally {
       deleteTable(table);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
index f805ca4..7ed8215 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
@@ -69,7 +69,7 @@ public class TestRecoverableZooKeeper {
 
   @Test
   public void testSetDataVersionMismatchInLoop() throws Exception {
-    String znode = "/hbase/region-in-transition/9af7cfc9b15910a0b3d714bf40a3248f";
+    String znode = "/hbase/splitWAL/9af7cfc9b15910a0b3d714bf40a3248f";
     Configuration conf = TEST_UTIL.getConfiguration();
     Properties properties = ZKConfig.makeZKProps(conf);
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSetDataVersionMismatchInLoop",


[12/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [8/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [8/8]

HBASE-11814 TestAssignmentManager.testCloseFailed() and testOpenCloseRacing() is flaky (Virag Kothari)


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

Branch: refs/heads/0.98
Commit: d487526888223a94bf37cc446cbf916fcd4a2535
Parents: 3ad7efd
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:02:14 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:04:28 2014 -0700

----------------------------------------------------------------------
 .../hbase/client/RpcRetryingCallerFactory.java  |   7 +-
 .../hadoop/hbase/master/RegionStateStore.java   |  41 ++---
 .../hadoop/hbase/util/MultiHConnection.java     | 163 +++++++++++++++++++
 .../master/TestAssignmentManagerOnCluster.java  |  62 +++++++
 4 files changed, 251 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d4875268/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
index 7957cc8..9b070a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java
@@ -47,9 +47,12 @@ public class RpcRetryingCallerFactory {
   }
 
   public static RpcRetryingCallerFactory instantiate(Configuration configuration) {
+    String clazzName = RpcRetryingCallerFactory.class.getName();
     String rpcCallerFactoryClazz =
-        configuration.get(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY,
-          RpcRetryingCallerFactory.class.getName());
+        configuration.get(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY, clazzName);
+    if (rpcCallerFactoryClazz.equals(clazzName)) {
+      return new RpcRetryingCallerFactory(configuration);
+    }
     return ReflectionUtils.instantiateWithCustomCtor(rpcCallerFactoryClazz,
       new Class[] { Configuration.class }, new Object[] { configuration });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d4875268/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
index d9f2262..c12a433 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,8 +32,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.master.RegionState.State;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConfigUtil;
+import org.apache.hadoop.hbase.util.MultiHConnection;
 
 import com.google.common.base.Preconditions;
 
@@ -52,12 +52,12 @@ public class RegionStateStore {
   private static final Log LOG = LogFactory.getLog(RegionStateStore.class);
 
   private volatile HRegion metaRegion;
-  private volatile HTableInterface metaTable;
   private volatile boolean initialized;
 
   private final boolean noPersistence;
   private final CatalogTracker catalogTracker;
   private final Server server;
+  private MultiHConnection multiHConnection;
 
   /**
    * Returns the {@link ServerName} from catalog table {@link Result}
@@ -113,16 +113,22 @@ public class RegionStateStore {
     initialized = false;
   }
 
-  @SuppressWarnings("deprecation")
   void start() throws IOException {
     if (!noPersistence) {
       if (server instanceof RegionServerServices) {
         metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
           HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
       }
+      // When meta is not colocated on master
       if (metaRegion == null) {
-        metaTable = new HTable(TableName.META_TABLE_NAME,
-          catalogTracker.getConnection());
+        Configuration conf = server.getConfiguration();
+        // Config to determine the no of HConnections to META.
+        // A single HConnection should be sufficient in most cases. Only if
+        // you are doing lot of writes (>1M) to META,
+        // increasing this value might improve the write throughput.
+        multiHConnection =
+            new MultiHConnection(conf, conf.getInt("hbase.regionstatestore.meta.connection", 1));
+
       }
     }
     initialized = true;
@@ -130,18 +136,11 @@ public class RegionStateStore {
 
   void stop() {
     initialized = false;
-    if (metaTable != null) {
-      try {
-        metaTable.close();
-      } catch (IOException e) {
-        LOG.info("Got exception in closing meta table", e);
-      } finally {
-        metaTable = null;
-      }
+    if (multiHConnection != null) {
+      multiHConnection.close();
     }
   }
 
-  @SuppressWarnings("deprecation")
   void updateRegionState(long openSeqNum,
       RegionState newState, RegionState oldState) {
     if (noPersistence || !initialized) {
@@ -182,6 +181,7 @@ public class RegionStateStore {
         Bytes.toBytes(state.name()));
       LOG.info(info);
 
+
       // Persist the state change to meta
       if (metaRegion != null) {
         try {
@@ -197,16 +197,17 @@ public class RegionStateStore {
           synchronized (this) {
             if (metaRegion != null) {
               LOG.info("Meta region shortcut failed", t);
-              metaTable = new HTable(TableName.META_TABLE_NAME,
-                catalogTracker.getConnection());
+              if (multiHConnection == null) {
+                multiHConnection = new MultiHConnection(server.getConfiguration(), 1);
+              }
               metaRegion = null;
             }
           }
         }
       }
-      synchronized(metaTable) {
-        metaTable.put(put);
-      }
+      // Called when meta is not on master
+      multiHConnection.processBatchCallback(Arrays.asList(put), TableName.META_TABLE_NAME, null,
+        null);
     } catch (IOException ioe) {
       LOG.error("Failed to persist region state " + newState, ioe);
       server.abort("Failed to update region location", ioe);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d4875268/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
new file mode 100644
index 0000000..26e222a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
@@ -0,0 +1,163 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+
+/**
+ * Provides ability to create multiple HConnection instances and allows to process a batch of
+ * actions using HConnection.processBatchCallback()
+ */
+@InterfaceAudience.Private
+public class MultiHConnection {
+  private static final Log LOG = LogFactory.getLog(MultiHConnection.class);
+  private HConnection[] hConnections;
+  private int noOfConnections;
+  private ExecutorService batchPool;
+
+  /**
+   * Create multiple HConnection instances and initialize a thread pool executor
+   * @param conf configuration
+   * @param noOfConnections total no of HConnections to create
+   * @throws IOException
+   */
+  public MultiHConnection(Configuration conf, int noOfConnections)
+      throws IOException {
+    this.noOfConnections = noOfConnections;
+    hConnections = new HConnection[noOfConnections];
+    for (int i = 0; i < noOfConnections; i++) {
+      HConnection conn = HConnectionManager.createConnection(conf);
+      hConnections[i] = conn;
+    }
+    createBatchPool(conf);
+  }
+
+  /**
+   * Close the open connections and shutdown the batchpool
+   */
+  public void close() {
+    if (hConnections != null) {
+      synchronized (hConnections) {
+        if (hConnections != null) {
+          for (HConnection conn : hConnections) {
+            if (conn != null) {
+              try {
+                conn.close();
+              } catch (IOException e) {
+                LOG.info("Got exception in closing connection", e);
+              } finally {
+                conn = null;
+              }
+            }
+          }
+          hConnections = null;
+        }
+      }
+    }
+    if (this.batchPool != null && !this.batchPool.isShutdown()) {
+      this.batchPool.shutdown();
+      try {
+        if (!this.batchPool.awaitTermination(10, TimeUnit.SECONDS)) {
+          this.batchPool.shutdownNow();
+        }
+      } catch (InterruptedException e) {
+        this.batchPool.shutdownNow();
+      }
+    }
+
+  }
+
+  private static ThreadLocal<Random> threadLocalRandom = new ThreadLocal<Random>() {
+    @Override
+    protected Random initialValue() {
+      return new Random();
+    }
+  };
+
+  /**
+   * Randomly pick a connection and process the batch of actions for a given table
+   * @param actions the actions
+   * @param tableName table name
+   * @param results the results array
+   * @param callback 
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @SuppressWarnings("deprecation")
+  public <R> void processBatchCallback(List<? extends Row> actions, TableName tableName,
+      Object[] results, Batch.Callback<R> callback) throws IOException {
+    // Currently used by RegionStateStore
+    // A deprecated method is used as multiple threads accessing RegionStateStore do a single put
+    // and htable is not thread safe. Alternative would be to create an Htable instance for each 
+    // put but that is not very efficient.
+    // See HBASE-11610 for more details.
+    try {
+      hConnections[threadLocalRandom.get().nextInt(noOfConnections)].processBatchCallback(
+        actions, tableName, this.batchPool, results, callback);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException(e.getMessage());
+    }
+  }
+
+  
+  // Copied from HConnectionImplementation.getBatchPool()
+  // We should get rid of this when HConnection.processBatchCallback is un-deprecated and provides
+  // an API to manage a batch pool
+  private void createBatchPool(Configuration conf) {
+    // Use the same config for keep alive as in HConnectionImplementation.getBatchPool();
+    int maxThreads = conf.getInt("hbase.multihconnection.threads.max", 256);
+    int coreThreads = conf.getInt("hbase.multihconnection.threads.core", 256);
+    if (maxThreads == 0) {
+      maxThreads = Runtime.getRuntime().availableProcessors() * 8;
+    }
+    if (coreThreads == 0) {
+      coreThreads = Runtime.getRuntime().availableProcessors() * 8;
+    }
+    long keepAliveTime = conf.getLong("hbase.multihconnection.threads.keepalivetime", 60);
+    LinkedBlockingQueue<Runnable> workQueue =
+        new LinkedBlockingQueue<Runnable>(maxThreads
+            * conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
+              HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
+    ThreadPoolExecutor tpe =
+        new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue,
+            Threads.newDaemonThreadFactory("MultiHConnection" + "-shared-"));
+    tpe.allowCoreThreadTimeOut(true);
+    this.batchPool = tpe;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d4875268/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index c20347e..c42128c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -47,8 +47,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -869,6 +871,66 @@ public class TestAssignmentManagerOnCluster {
     }
   }
 
+  /**
+   * Test concurrent updates to meta when meta is not on master. Only for zk-less assignment
+   * @throws Exception
+   */
+  @Test(timeout = 30000)
+  public void testUpdatesRemoteMeta() throws Exception {
+    // Not for zk less assignment
+    if (conf.getBoolean("hbase.assignment.usezk", true)) {
+      return;
+    }
+    conf.setInt("hbase.regionstatestore.meta.connection", 3);
+    final RegionStateStore rss = new RegionStateStore(new MyRegionServer(conf));
+    rss.start();
+    // Create 10 threads and make each do 10 puts related to region state update
+    Thread[] th = new Thread[10];
+    List<String> nameList = new ArrayList<String>();
+    List<TableName> tableNameList = new ArrayList<TableName>();
+    for (int i = 0; i < th.length; i++) {
+      th[i] = new Thread() {
+        @Override
+        public void run() {
+          HRegionInfo[] hri = new HRegionInfo[10];
+          ServerName serverName = ServerName.valueOf("dummyhost", 1000, 1234);
+          for (int i = 0; i < 10; i++) {
+            hri[i] = new HRegionInfo(TableName.valueOf(Thread.currentThread().getName() + "_" + i));
+            RegionState newState = new RegionState(hri[i], RegionState.State.OPEN, serverName);
+            RegionState oldState =
+                new RegionState(hri[i], RegionState.State.PENDING_OPEN, serverName);
+            rss.updateRegionState(1, newState, oldState);
+          }
+        }
+      };
+      th[i].start();
+      nameList.add(th[i].getName());
+    }
+    for (int i = 0; i < th.length; i++) {
+      th[i].join();
+    }
+    // Add all the expected table names in meta to tableNameList
+    for (String name : nameList) {
+      for (int i = 0; i < 10; i++) {
+        tableNameList.add(TableName.valueOf(name + "_" + i));
+      }
+    }
+    List<Result> metaRows =
+        MetaReader.fullScan(TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
+    int count = 0;
+    // Check all 100 rows are in meta
+    for (Result result : metaRows) {
+      if (tableNameList.contains(HRegionInfo.getTable(result.getRow()))) {
+        count++;
+        if (count == 100) {
+          break;
+        }
+      }
+    }
+    assertTrue(count == 100);
+    rss.stop();
+  }
+
   static class MyLoadBalancer extends StochasticLoadBalancer {
     // For this region, if specified, always assign to nowhere
     static volatile String controledRegion = null;


[11/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [7/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [7/8]

HBASE-11615 TestZKLessAMOnCluster.testForceAssignWhileClosing failed on Jenkins (Jimmy Xiang)


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

Branch: refs/heads/0.98
Commit: 3ad7efd8c3b130fecc1d1dbef6099a08e97e2a22
Parents: e09adb8
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:04:17 2014 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3ad7efd8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index c3a507d..c20347e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -380,6 +380,8 @@ public class TestAssignmentManagerOnCluster {
       master.assignRegion(hri);
       AssignmentManager am = master.getAssignmentManager();
       assertTrue(am.waitForAssignment(hri));
+      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
       
       MyRegionObserver.preCloseEnabled.set(true);
       am.unassign(hri);


[06/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [2/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [2/8]

HBASE-11687 No need to abort on postOpenDeployTasks exception if region opening is cancelled (Jimmy Xiang)


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

Branch: refs/heads/0.98
Commit: cbc7860690c355433b305523b4c1f2f1319432e8
Parents: e6ffa86
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:03:23 2014 -0700

----------------------------------------------------------------------
 .../regionserver/handler/OpenRegionHandler.java | 26 +++++++++++++-------
 1 file changed, 17 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cbc78606/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
index ec78aa4..061d25c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
@@ -318,7 +318,7 @@ public class OpenRegionHandler extends EventHandler {
    * .
    */
   static class PostOpenDeployTasksThread extends Thread {
-    private Exception exception = null;
+    private Throwable exception = null;
     private final Server server;
     private final RegionServerServices services;
     private final HRegion region;
@@ -338,13 +338,16 @@ public class OpenRegionHandler extends EventHandler {
       try {
         this.services.postOpenDeployTasks(this.region,
           this.server.getCatalogTracker());
-      } catch (KeeperException e) {
-        server.abort("Exception running postOpenDeployTasks; region=" +
-            this.region.getRegionInfo().getEncodedName(), e);
-      } catch (Exception e) {
-        LOG.warn("Exception running postOpenDeployTasks; region=" +
-          this.region.getRegionInfo().getEncodedName(), e);
+      } catch (Throwable e) {
+        String msg =
+            "Exception running postOpenDeployTasks; region="
+                + this.region.getRegionInfo().getEncodedName();
         this.exception = e;
+        if (e instanceof IOException && isRegionStillOpening(region.getRegionInfo(), services)) {
+          server.abort(msg, e);
+        } else {
+          LOG.warn(msg, e);
+        }
       }
       // We're done.  Set flag then wake up anyone waiting on thread to complete.
       this.signaller.set(true);
@@ -356,7 +359,7 @@ public class OpenRegionHandler extends EventHandler {
     /**
      * @return Null or the run exception; call this method after thread is done.
      */
-    Exception getException() {
+    Throwable getException() {
       return this.exception;
     }
   }
@@ -522,12 +525,17 @@ public class OpenRegionHandler extends EventHandler {
     }
   }
 
-  private boolean isRegionStillOpening() {
+  private static boolean isRegionStillOpening(HRegionInfo regionInfo,
+      RegionServerServices rsServices) {
     byte[] encodedName = regionInfo.getEncodedNameAsBytes();
     Boolean action = rsServices.getRegionsInTransitionInRS().get(encodedName);
     return Boolean.TRUE.equals(action); // true means opening for RIT
   }
 
+  private boolean isRegionStillOpening() {
+    return isRegionStillOpening(regionInfo, rsServices);
+  }
+
   /**
    * Transition ZK node from OFFLINE to OPENING.
    * @param encodedName Name of the znode file (Region encodedName is the znode


[03/12] HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-protocol/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol/src/main/protobuf/RegionServerStatus.proto
index a3b4d74..fbd6e05 100644
--- a/hbase-protocol/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol/src/main/protobuf/RegionServerStatus.proto
@@ -78,6 +78,47 @@ message GetLastFlushedSequenceIdResponse {
   required uint64 last_flushed_sequence_id = 1;
 }
 
+message RegionTransition {
+  required TransitionCode transition_code = 1;
+
+  /** Mutliple regions are involved during merging/splitting */
+  repeated RegionInfo region_info = 2;
+
+  /** For newly opened region, the open seq num is needed */
+  optional uint64 open_seq_num = 3;
+
+  enum TransitionCode {
+    OPENED = 0;
+    FAILED_OPEN = 1;
+    /** No failed_close, in which case region server will abort */
+    CLOSED = 2;
+
+    /** Ask master for ok to split/merge region(s) */
+    READY_TO_SPLIT = 3;
+    READY_TO_MERGE = 4;
+
+    SPLIT_PONR = 5;
+    MERGE_PONR = 6;
+
+    SPLIT = 7;
+    MERGED = 8;
+    SPLIT_REVERTED = 9;
+    MERGE_REVERTED = 10;
+  }
+}
+
+message ReportRegionTransitionRequest {
+  /** This region server's server name */
+  required ServerName server = 1;
+
+  repeated RegionTransition transition = 2;
+}
+
+message ReportRegionTransitionResponse {
+  /** Error message if failed to update the region state */
+  optional string error_message = 1;
+}
+
 service RegionServerStatusService {
   /** Called when a region server first starts. */
   rpc RegionServerStartup(RegionServerStartupRequest)
@@ -99,4 +140,12 @@ service RegionServerStatusService {
    * log splitting. */
   rpc GetLastFlushedSequenceId(GetLastFlushedSequenceIdRequest)
     returns(GetLastFlushedSequenceIdResponse);
+
+  /**
+   * Called by a region server to report the progress of a region
+   * transition. If the request fails, the transition should
+   * be aborted.
+   */
+  rpc ReportRegionTransition(ReportRegionTransitionRequest)
+    returns(ReportRegionTransitionResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
index adeeed4..ddab430 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
@@ -34,15 +34,18 @@ public class AssignCallable implements Callable<Object> {
   private AssignmentManager assignmentManager;
 
   private HRegionInfo hri;
+  private boolean newPlan;
 
-  public AssignCallable(AssignmentManager assignmentManager, HRegionInfo hri) {
+  public AssignCallable(
+      AssignmentManager assignmentManager, HRegionInfo hri, boolean newPlan) {
     this.assignmentManager = assignmentManager;
+    this.newPlan = newPlan;
     this.hri = hri;
   }
 
   @Override
   public Object call() throws Exception {
-    assignmentManager.assign(hri, true, true);
+    assignmentManager.assign(hri, true, newPlan);
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 5e3c4c8..b6fd2cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -44,6 +44,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
@@ -73,12 +75,19 @@ import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
 import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
 import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.regionserver.SplitTransaction;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -96,6 +105,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.data.Stat;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.LinkedHashMultimap;
 
@@ -237,6 +247,13 @@ public class AssignmentManager extends ZooKeeperListener {
   private final ConcurrentHashMap<String, AtomicInteger>
     failedOpenTracker = new ConcurrentHashMap<String, AtomicInteger>();
 
+  // A flag to indicate if we are using ZK for region assignment
+  private final boolean useZKForAssignment;
+
+  // In case not using ZK for region assignment, region states
+  // are persisted in meta with a state store
+  private final RegionStateStore regionStateStore;
+
   /**
    * For testing only!  Set to true to skip handling of split.
    */
@@ -265,6 +282,7 @@ public class AssignmentManager extends ZooKeeperListener {
     this.serverManager = serverManager;
     this.catalogTracker = catalogTracker;
     this.executorService = service;
+    this.regionStateStore = new RegionStateStore(server);
     this.regionsToReopen = Collections.synchronizedMap
                            (new HashMap<String, HRegionInfo> ());
     Configuration conf = server.getConfiguration();
@@ -299,7 +317,7 @@ public class AssignmentManager extends ZooKeeperListener {
     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
-    this.regionStates = new RegionStates(server, serverManager);
+    this.regionStates = new RegionStates(server, serverManager, regionStateStore);
 
     this.bulkAssignWaitTillAllAssigned =
       conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
@@ -313,6 +331,7 @@ public class AssignmentManager extends ZooKeeperListener {
     this.tableLockManager = tableLockManager;
 
     this.metricsAssignmentManager = new MetricsAssignmentManager();
+    useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
   }
 
   void startTimeOutMonitor() {
@@ -357,6 +376,14 @@ public class AssignmentManager extends ZooKeeperListener {
     return regionStates;
   }
 
+  /**
+   * Used in some tests to mock up region state in meta
+   */
+  @VisibleForTesting
+  RegionStateStore getRegionStateStore() {
+    return regionStateStore;
+  }
+
   public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
     return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
   }
@@ -452,6 +479,7 @@ public class AssignmentManager extends ZooKeeperListener {
    */
   void joinCluster() throws IOException,
       KeeperException, InterruptedException {
+    long startTime = System.currentTimeMillis();
     // Concurrency note: In the below the accesses on regionsInTransition are
     // outside of a synchronization block where usually all accesses to RIT are
     // synchronized.  The presumption is that in this case it is safe since this
@@ -467,10 +495,16 @@ public class AssignmentManager extends ZooKeeperListener {
     // This method will assign all user regions if a clean server startup or
     // it will reconstruct master state and cleanup any leftovers from
     // previous master process.
-    processDeadServersAndRegionsInTransition(deadServers);
+    boolean failover = processDeadServersAndRegionsInTransition(deadServers);
 
+    if (!useZKForAssignment) {
+      // Not use ZK for assignment any more, remove the ZNode
+      ZKUtil.deleteNodeFailSilent(watcher, watcher.assignmentZNode);
+    }
     recoverTableInDisablingState();
     recoverTableInEnablingState();
+    LOG.info("Joined the cluster in " + (System.currentTimeMillis()
+      - startTime) + "ms, failover=" + failover);
   }
 
   /**
@@ -484,37 +518,56 @@ public class AssignmentManager extends ZooKeeperListener {
    * @throws IOException
    * @throws InterruptedException
    */
-  void processDeadServersAndRegionsInTransition(
+  boolean processDeadServersAndRegionsInTransition(
       final Map<ServerName, List<HRegionInfo>> deadServers)
           throws KeeperException, IOException, InterruptedException {
     List<String> nodes = ZKUtil.listChildrenNoWatch(watcher,
       watcher.assignmentZNode);
 
-    if (nodes == null) {
+    if (nodes == null && useZKForAssignment) {
       String errorMessage = "Failed to get the children from ZK";
       server.abort(errorMessage, new IOException(errorMessage));
-      return;
+      return true; // Doesn't matter in this case
     }
 
-    boolean failover = (!serverManager.getDeadServers().isEmpty() || !serverManager
-        .getRequeuedDeadServers().isEmpty());
-
-    if (!failover) {
+    boolean failover = !serverManager.getDeadServers().isEmpty();
+    if (failover) {
+      // This may not be a failover actually, especially if meta is on this master.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found dead servers out on cluster " + serverManager.getDeadServers());
+      }
+    } else {
       // If any one region except meta is assigned, it's a failover.
-      Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
-      for (HRegionInfo hri: regions.keySet()) {
+      for (HRegionInfo hri : regionStates.getRegionAssignments().keySet()) {
         if (!hri.isMetaTable()) {
           LOG.debug("Found " + hri + " out on cluster");
           failover = true;
           break;
         }
       }
-      if (!failover) {
-        // If any one region except meta is in transition, it's a failover.
-        for (String encodedName: nodes) {
-          RegionState state = regionStates.getRegionState(encodedName);
-          if (state != null && !state.getRegion().isMetaRegion()) {
-            LOG.debug("Found " + state.getRegion().getRegionNameAsString() + " in RITs");
+    }
+
+    if (!failover && nodes != null) {
+      // If any one region except meta is in transition, it's a failover.
+      for (String encodedName : nodes) {
+        RegionState regionState = regionStates.getRegionState(encodedName);
+        if (regionState != null && !regionState.getRegion().isMetaRegion()) {
+          LOG.debug("Found " + regionState + " in RITs");
+          failover = true;
+          break;
+        }
+      }
+    }
+
+    if (!failover && !useZKForAssignment) {
+      // If any region except meta is in transition on a live server, it's a failover.
+      Map<String, RegionState> regionsInTransition = regionStates.getRegionsInTransition();
+      if (!regionsInTransition.isEmpty()) {
+        Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
+        for (RegionState regionState : regionsInTransition.values()) {
+          if (!regionState.getRegion().isMetaRegion()
+              && onlineServers.contains(regionState.getServerName())) {
+            LOG.debug("Found " + regionState + " in RITs");
             failover = true;
             break;
           }
@@ -522,17 +575,70 @@ public class AssignmentManager extends ZooKeeperListener {
       }
     }
 
+    if (!failover) {
+      // If we get here, we have a full cluster restart. It is a failover only
+      // if there are some HLogs are not split yet. For meta HLogs, they should have
+      // been split already, if any. We can walk through those queued dead servers,
+      // if they don't have any HLogs, this restart should be considered as a clean one
+      Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
+      if (!queuedDeadServers.isEmpty()) {
+        Configuration conf = server.getConfiguration();
+        Path rootdir = FSUtils.getRootDir(conf);
+        FileSystem fs = rootdir.getFileSystem(conf);
+        for (ServerName serverName : queuedDeadServers) {
+          Path logDir = new Path(rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
+          Path splitDir = logDir.suffix(HLog.SPLITTING_EXT);
+          if (fs.exists(logDir) || fs.exists(splitDir)) {
+            LOG.debug("Found queued dead server " + serverName);
+            failover = true;
+            break;
+          }
+        }
+        if (!failover) {
+          // We figured that it's not a failover, so no need to
+          // work on these re-queued dead servers any more.
+          LOG.info("AM figured that it's not a failover and cleaned up " + queuedDeadServers.size()
+              + " queued dead servers");
+          serverManager.removeRequeuedDeadServers();
+        }
+      }
+    }
+
+    Set<TableName> disabledOrDisablingOrEnabling = null;
+    if (!failover) {
+      disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
+      disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
+      // Clean re/start, mark all user regions closed before reassignment
+      // TODO -Hbase-11319
+      regionStates.closeAllUserRegions(disabledOrDisablingOrEnabling);
+    }
+
+    // Now region states are restored
+    regionStateStore.start();
+
     // If we found user regions out on cluster, its a failover.
     if (failover) {
       LOG.info("Found regions out on cluster or in RIT; presuming failover");
       // Process list of dead servers and regions in RIT.
       // See HBASE-4580 for more information.
       processDeadServersAndRecoverLostRegions(deadServers);
-    } else {
+    } 
+    if (!failover && useZKForAssignment) {
+      // Cleanup any existing ZK nodes and start watching
+      ZKAssign.deleteAllNodes(watcher);
+      ZKUtil.listChildrenAndWatchForNewChildren(this.watcher, this.watcher.assignmentZNode);
+    }
+    // Now we can safely claim failover cleanup completed and enable
+    // ServerShutdownHandler for further processing. The nodes (below)
+    // in transition, if any, are for regions not related to those
+    // dead servers at all, and can be done in parallel to SSH.
+    failoverCleanupDone();
+    if (!failover) {
       // Fresh cluster startup.
-      LOG.info("Clean cluster startup. Assigning userregions");
-      assignAllUserRegions();
+      LOG.info("Clean cluster startup. Assigning user regions");
+      assignAllUserRegions(disabledOrDisablingOrEnabling);
     }
+    return failover;
   }
 
   /**
@@ -684,7 +790,7 @@ public class AssignmentManager extends ZooKeeperListener {
             public void process() throws IOException {
               ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
               try {
-                unassign(regionInfo, rsClosing, expectedVersion, null, true, null);
+                unassign(regionInfo, rsClosing, expectedVersion, null, useZKForAssignment, null);
                 if (regionStates.isRegionOffline(regionInfo)) {
                   assign(regionInfo, true);
                 }
@@ -1282,6 +1388,7 @@ public class AssignmentManager extends ZooKeeperListener {
     }
   }
 
+  
   /**
    * Marks the region as online.  Removes it from regions in transition and
    * updates the in-memory assignment information.
@@ -1291,8 +1398,12 @@ public class AssignmentManager extends ZooKeeperListener {
    * @param sn
    */
   void regionOnline(HRegionInfo regionInfo, ServerName sn) {
+    regionOnline(regionInfo, sn, HConstants.NO_SEQNUM);
+  }
+
+  void regionOnline(HRegionInfo regionInfo, ServerName sn, long openSeqNum) {
     numRegionsOpened.incrementAndGet();
-    regionStates.regionOnline(regionInfo, sn);
+    regionStates.regionOnline(regionInfo, sn, openSeqNum);
 
     // Remove plan if one.
     clearRegionPlan(regionInfo);
@@ -1400,13 +1511,15 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   public void offlineDisabledRegion(HRegionInfo regionInfo) {
-    // Disabling so should not be reassigned, just delete the CLOSED node
-    LOG.debug("Table being disabled so deleting ZK node and removing from " +
-      "regions in transition, skipping assignment of region " +
-        regionInfo.getRegionNameAsString());
-    String encodedName = regionInfo.getEncodedName();
-    deleteNodeInStates(encodedName, "closed", null,
-      EventType.RS_ZK_REGION_CLOSED, EventType.M_ZK_REGION_OFFLINE);
+    if (useZKForAssignment) {
+      // Disabling so should not be reassigned, just delete the CLOSED node
+      LOG.debug("Table being disabled so deleting ZK node and removing from " +
+        "regions in transition, skipping assignment of region " +
+          regionInfo.getRegionNameAsString());
+      String encodedName = regionInfo.getEncodedName();
+      deleteNodeInStates(encodedName, "closed", null,
+        EventType.RS_ZK_REGION_CLOSED, EventType.M_ZK_REGION_OFFLINE);
+    }
     regionOffline(regionInfo);
   }
 
@@ -1458,7 +1571,7 @@ public class AssignmentManager extends ZooKeeperListener {
             + " is dead but not processed yet");
           return;
         }
-        assign(state, setOfflineInZK, forceNewPlan);
+        assign(state, setOfflineInZK && useZKForAssignment, forceNewPlan);
       }
     } finally {
       lock.unlock();
@@ -1504,7 +1617,8 @@ public class AssignmentManager extends ZooKeeperListener {
                   + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
                   + " is dead but not processed yet");
                 onDeadServer = true;
-              } else if (asyncSetOfflineInZooKeeper(state, cb, destination)) {
+              } else if (!useZKForAssignment
+                  || asyncSetOfflineInZooKeeper(state, cb, destination)) {
                 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
                 plans.put(encodedName, plan);
                 states.add(state);
@@ -1523,18 +1637,19 @@ public class AssignmentManager extends ZooKeeperListener {
           Lock lock = locks.remove(encodedName);
           lock.unlock();
         }
-
-        // Wait until all unassigned nodes have been put up and watchers set.
-        int total = states.size();
-        for (int oldCounter = 0; !server.isStopped();) {
-          int count = counter.get();
-          if (oldCounter != count) {
-            LOG.info(destination.toString() + " unassigned znodes=" + count +
-              " of total=" + total);
-            oldCounter = count;
+        if (useZKForAssignment) {
+          // Wait until all unassigned nodes have been put up and watchers set.
+          int total = states.size();
+          for (int oldCounter = 0; !server.isStopped();) {
+            int count = counter.get();
+            if (oldCounter != count) {
+              LOG.info(destination.toString() + " unassigned znodes=" + count + " of total="
+                  + total);
+              oldCounter = count;
+            }
+            if (count >= total) break;
+            Threads.sleep(5);
           }
-          if (count >= total) break;
-          Threads.sleep(5);
         }
 
         if (server.isStopped()) {
@@ -1551,7 +1666,7 @@ public class AssignmentManager extends ZooKeeperListener {
           HRegionInfo region = state.getRegion();
           String encodedRegionName = region.getEncodedName();
           Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
-          if (nodeVersion == null || nodeVersion == -1) {
+          if (useZKForAssignment && (nodeVersion == null || nodeVersion == -1)) {
             LOG.warn("failed to offline in zookeeper: " + region);
             failedToOpenRegions.add(region); // assign individually later
             Lock lock = locks.remove(encodedRegionName);
@@ -1706,11 +1821,11 @@ public class AssignmentManager extends ZooKeeperListener {
           versionOfClosingNode, dest, transitionInZK)) {
           LOG.debug("Sent CLOSE to " + server + " for region " +
             region.getRegionNameAsString());
-          if (!transitionInZK && state != null) {
+          if (useZKForAssignment && !transitionInZK && state != null) {
             // Retry to make sure the region is
             // closed so as to avoid double assignment.
             unassign(region, state, versionOfClosingNode,
-              dest, transitionInZK,src);
+              dest, transitionInZK, src);
           }
           return;
         }
@@ -1834,10 +1949,14 @@ public class AssignmentManager extends ZooKeeperListener {
       // yet, we can move on only if the meta shows the
       // region is not on this server actually, or on a server
       // not dead, or dead and processed already.
-      if (regionStates.isServerDeadAndNotProcessed(sn)
+      // In case not using ZK, we don't need this check because
+      // we have the latest info in memory, and the caller
+      // will do another round checking any way.
+      if (useZKForAssignment
+          && regionStates.isServerDeadAndNotProcessed(sn)
           && wasRegionOnDeadServerByMeta(region, sn)) {
         LOG.info("Skip assigning " + region.getRegionNameAsString()
-          + ", it is on a dead but not processed yet server");
+          + ", it is on a dead but not processed yet server: " + sn);
         return null;
       }
     case CLOSED:
@@ -1860,9 +1979,9 @@ public class AssignmentManager extends ZooKeeperListener {
       while (!server.isStopped()) {
         try {
           catalogTracker.waitForMeta();
-          Pair<HRegionInfo, ServerName> r =
-            MetaReader.getRegion(catalogTracker, region.getRegionName());
-          ServerName server = r == null ? null : r.getSecond();
+          Result r = MetaReader.getRegionResult(catalogTracker, region.getRegionName());
+          if (r == null || r.isEmpty()) return false;
+          ServerName server = HRegionInfo.getServerName(r);
           return regionStates.isServerDeadAndNotProcessed(server);
         } catch (IOException ioe) {
           LOG.info("Received exception accessing hbase:meta during force assign "
@@ -2326,14 +2445,16 @@ public class AssignmentManager extends ZooKeeperListener {
             regionOffline(region);
             return;
           }
-          versionOfClosingNode = ZKAssign.createNodeClosing(
-            watcher, region, state.getServerName());
-          if (versionOfClosingNode == -1) {
-            LOG.info("Attempting to unassign " +
-              region.getRegionNameAsString() + " but ZK closing node "
-              + "can't be created.");
-            reassign = false; // not unassigned at all
-            return;
+          if (useZKForAssignment) {
+            versionOfClosingNode = ZKAssign.createNodeClosing(
+              watcher, region, state.getServerName());
+            if (versionOfClosingNode == -1) {
+              LOG.info("Attempting to unassign " +
+                region.getRegionNameAsString() + " but ZK closing node "
+                + "can't be created.");
+              reassign = false; // not unassigned at all
+              return;
+            }
           }
         } catch (KeeperException e) {
           if (e instanceof NodeExistsException) {
@@ -2386,7 +2507,7 @@ public class AssignmentManager extends ZooKeeperListener {
         return;
       }
 
-      unassign(region, state, versionOfClosingNode, dest, true, null);
+      unassign(region, state, versionOfClosingNode, dest, useZKForAssignment, null);
     } finally {
       lock.unlock();
 
@@ -2590,26 +2711,20 @@ public class AssignmentManager extends ZooKeeperListener {
    * @throws IOException
    * @throws KeeperException
    */
-  private void assignAllUserRegions()
+  private void assignAllUserRegions(Set<TableName> disabledOrDisablingOrEnabling)
       throws IOException, InterruptedException, KeeperException {
-    // Cleanup any existing ZK nodes and start watching
-    ZKAssign.deleteAllNodes(watcher);
-    ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
-      this.watcher.assignmentZNode);
-    failoverCleanupDone();
-
     // Skip assignment for regions of tables in DISABLING state because during clean cluster startup
     // no RS is alive and regions map also doesn't have any information about the regions.
     // See HBASE-6281.
-    Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
-    disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
     // Scan hbase:meta for all user regions, skipping any disabled tables
     Map<HRegionInfo, ServerName> allRegions;
     SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
        new SnapshotOfRegionAssignmentFromMeta(catalogTracker, disabledOrDisablingOrEnabling, true);
     snapshotOfRegionAssignment.initialize();
     allRegions = snapshotOfRegionAssignment.getRegionToRegionServerMap();
-    if (allRegions == null || allRegions.isEmpty()) return;
+    if (allRegions == null || allRegions.isEmpty()) {
+      return;
+    }
 
     // Determine what type of assignment to do on startup
     boolean retainAssignment = server.getConfiguration().
@@ -2679,65 +2794,44 @@ public class AssignmentManager extends ZooKeeperListener {
       new TreeMap<ServerName, List<HRegionInfo>>();
     // Iterate regions in META
     for (Result result : results) {
-      Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(result);
-      if (region == null) continue;
-      HRegionInfo regionInfo = region.getFirst();
-      ServerName regionLocation = region.getSecond();
+      HRegionInfo regionInfo = HRegionInfo.getHRegionInfo(result);
       if (regionInfo == null) continue;
-      regionStates.createRegionState(regionInfo);
-      if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
-        // Split is considered to be completed. If the split znode still
-        // exists, the region will be put back to SPLITTING state later
-        LOG.debug("Region " + regionInfo.getRegionNameAsString()
-           + " split is completed. Hence need not add to regions list");
+      State state = RegionStateStore.getRegionState(result);
+      ServerName regionLocation = RegionStateStore.getRegionServer(result);
+      regionStates.createRegionState(regionInfo, state, regionLocation);
+      if (!regionStates.isRegionInState(regionInfo, State.OPEN)) {
+        // Region is not open (either offline or in transition), skip
         continue;
       }
       TableName tableName = regionInfo.getTable();
-      if (regionLocation == null) {
-        // regionLocation could be null if createTable didn't finish properly.
-        // When createTable is in progress, HMaster restarts.
-        // Some regions have been added to hbase:meta, but have not been assigned.
-        // When this happens, the region's table must be in ENABLING state.
-        // It can't be in ENABLED state as that is set when all regions are
-        // assigned.
-        // It can't be in DISABLING state, because DISABLING state transitions
-        // from ENABLED state when application calls disableTable.
-        // It can't be in DISABLED state, because DISABLED states transitions
-        // from DISABLING state.
-        if (!enablingTables.contains(tableName)) {
-          LOG.warn("Region " + regionInfo.getEncodedName() +
-            " has null regionLocation." + " But its table " + tableName +
-            " isn't in ENABLING state.");
-        }
-      } else if (!onlineServers.contains(regionLocation)) {
+      if (!onlineServers.contains(regionLocation)) {
         // Region is located on a server that isn't online
         List<HRegionInfo> offlineRegions = offlineServers.get(regionLocation);
         if (offlineRegions == null) {
           offlineRegions = new ArrayList<HRegionInfo>(1);
           offlineServers.put(regionLocation, offlineRegions);
         }
-        offlineRegions.add(regionInfo);
-        // need to enable the table if not disabled or disabling or enabling
-        // this will be used in rolling restarts
-        if (!disabledOrDisablingOrEnabling.contains(tableName)
-            && !getZKTable().isEnabledTable(tableName)) {
-          setEnabledTable(tableName);
+        if (useZKForAssignment) {
+          regionStates.regionOffline(regionInfo);
         }
-      } else {
+        offlineRegions.add(regionInfo);
+      } else if (!disabledOrEnablingTables.contains(tableName)) {
         // Region is being served and on an active server
         // add only if region not in disabled or enabling table
-        if (!disabledOrEnablingTables.contains(tableName)) {
-          regionStates.updateRegionState(regionInfo, State.OPEN, regionLocation);
-          regionStates.regionOnline(regionInfo, regionLocation);
-          balancer.regionOnline(regionInfo, regionLocation);
-        }
-        // need to enable the table if not disabled or disabling or enabling
-        // this will be used in rolling restarts
-        if (!disabledOrDisablingOrEnabling.contains(tableName)
-            && !getZKTable().isEnabledTable(tableName)) {
-          setEnabledTable(tableName);
-        }
+
+        regionStates.updateRegionState(regionInfo, State.OPEN, regionLocation);
+        regionStates.regionOnline(regionInfo, regionLocation);
+        balancer.regionOnline(regionInfo, regionLocation);
+      } else if (useZKForAssignment) {
+        regionStates.regionOffline(regionInfo);
       }
+      // need to enable the table if not disabled or disabling or enabling
+      // this will be used in rolling restarts
+      if (!disabledOrDisablingOrEnabling.contains(tableName)
+          && !getZKTable().isEnabledTable(tableName)) {
+        setEnabledTable(tableName);
+      }
+
     }
     return offlineServers;
   }
@@ -2825,19 +2919,146 @@ public class AssignmentManager extends ZooKeeperListener {
         }
       }
     }
-    List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(
-      this.watcher, this.watcher.assignmentZNode);
-    if (!nodes.isEmpty()) {
+
+    List<String> nodes = useZKForAssignment ?
+      ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode)
+      : ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
+    if (nodes != null && !nodes.isEmpty()) {
       for (String encodedRegionName : nodes) {
         processRegionInTransition(encodedRegionName, null);
       }
+    } else if (!useZKForAssignment) {
+      // We need to send RPC call again for PENDING_OPEN/PENDING_CLOSE regions
+      // in case the RPC call is not sent out yet before the master was shut down
+      // since we update the state before we send the RPC call. We can't update
+      // the state after the RPC call. Otherwise, we don't know what's happened
+      // to the region if the master dies right after the RPC call is out.
+      Map<String, RegionState> rits = regionStates.getRegionsInTransition();
+      for (RegionState regionState: rits.values()) {
+        if (!serverManager.isServerOnline(regionState.getServerName())) {
+          continue; // SSH will handle it
+        }
+        State state = regionState.getState();
+        LOG.info("Processing " + regionState);
+        switch (state) {
+        case PENDING_OPEN:
+          retrySendRegionOpen(regionState);
+          break;
+        case PENDING_CLOSE:
+          retrySendRegionClose(regionState);
+          break;
+        default:
+          // No process for other states
+        }
+      }
     }
+  }
 
-    // Now we can safely claim failover cleanup completed and enable
-    // ServerShutdownHandler for further processing. The nodes (below)
-    // in transition, if any, are for regions not related to those
-    // dead servers at all, and can be done in parallel to SSH.
-    failoverCleanupDone();
+  /**
+   * At master failover, for pending_open region, make sure
+   * sendRegionOpen RPC call is sent to the target regionserver
+   */
+  private void retrySendRegionOpen(final RegionState regionState) {
+    this.executorService.submit(
+      new EventHandler(server, EventType.M_MASTER_RECOVERY) {
+        @Override
+        public void process() throws IOException {
+          HRegionInfo hri = regionState.getRegion();
+          ServerName serverName = regionState.getServerName();
+          ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
+          try {
+            while (serverManager.isServerOnline(serverName)
+                && !server.isStopped() && !server.isAborted()) {
+              try {
+                List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
+                if (shouldAssignRegionsWithFavoredNodes) {
+                  favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri);
+                }
+                RegionOpeningState regionOpenState = serverManager.sendRegionOpen(
+                  serverName, hri, -1, favoredNodes);
+
+                if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
+                  // Failed opening this region, this means the target server didn't get
+                  // the original region open RPC, so re-assign it with a new plan
+                  LOG.debug("Got failed_opening in retry sendRegionOpen for "
+                    + regionState + ", re-assign it");
+                  invokeAssign(hri, true);
+                }
+                return; // Done.
+              } catch (Throwable t) {
+                if (t instanceof RemoteException) {
+                  t = ((RemoteException) t).unwrapRemoteException();
+                }
+                // In case SocketTimeoutException/FailedServerException, we will retry
+                if (t instanceof java.net.SocketTimeoutException
+                    || t instanceof FailedServerException) {
+                  Threads.sleep(100);
+                  continue;
+                }
+                // For other exceptions, re-assign it
+                LOG.debug("Got exception in retry sendRegionOpen for "
+                  + regionState + ", re-assign it", t);
+                invokeAssign(hri);
+                return; // Done.
+              }
+            }
+          } finally {
+            lock.unlock();
+          }
+        }
+      });
+  }
+
+  /**
+   * At master failover, for pending_close region, make sure
+   * sendRegionClose RPC call is sent to the target regionserver
+   */
+  private void retrySendRegionClose(final RegionState regionState) {
+    this.executorService.submit(
+      new EventHandler(server, EventType.M_MASTER_RECOVERY) {
+        @Override
+        public void process() throws IOException {
+          HRegionInfo hri = regionState.getRegion();
+          ServerName serverName = regionState.getServerName();
+          ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
+          try {
+            while (serverManager.isServerOnline(serverName)
+                && !server.isStopped() && !server.isAborted()) {
+              try {
+                if (!serverManager.sendRegionClose(serverName, hri, -1, null, false)) {
+                  // This means the region is still on the target server
+                  LOG.debug("Got false in retry sendRegionClose for "
+                    + regionState + ", re-close it");
+                  invokeUnAssign(hri);
+                }
+                return; // Done.
+              } catch (Throwable t) {
+                if (t instanceof RemoteException) {
+                  t = ((RemoteException) t).unwrapRemoteException();
+                }
+                // In case SocketTimeoutException/FailedServerException, we will retry
+                if (t instanceof java.net.SocketTimeoutException
+                    || t instanceof FailedServerException) {
+                  Threads.sleep(100);
+                  continue;
+                }
+                if (!(t instanceof NotServingRegionException
+                    || t instanceof RegionAlreadyInTransitionException)) {
+                  // NotServingRegionException/RegionAlreadyInTransitionException
+                  // means the target server got the original region close request.
+                  // For other exceptions, re-close it
+                  LOG.debug("Got exception in retry sendRegionClose for "
+                    + regionState + ", re-close it", t);
+                  invokeUnAssign(hri);
+                }
+                return; // Done.
+              }
+            }
+          } finally {
+            lock.unlock();
+          }
+        }
+      });
   }
 
   /**
@@ -3098,7 +3319,15 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   void invokeAssign(HRegionInfo regionInfo) {
-    threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
+    invokeAssign(regionInfo, true);
+  }
+
+  void invokeAssign(HRegionInfo regionInfo, boolean newPlan) {
+    threadPoolExecutorService.submit(new AssignCallable(this, regionInfo, newPlan));
+  }
+
+  void invokeUnAssign(HRegionInfo regionInfo) {
+    threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
   }
 
   private void invokeUnassign(HRegionInfo regionInfo) {
@@ -3257,6 +3486,7 @@ public class AssignmentManager extends ZooKeeperListener {
     }
     threadPoolExecutorService.shutdownNow();
     zkEventWorkers.shutdownNow();
+    regionStateStore.stop();
   }
 
   protected void setEnabledTable(TableName tableName) {
@@ -3330,6 +3560,190 @@ public class AssignmentManager extends ZooKeeperListener {
       EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
   }
 
+  private void onRegionFailedOpen(
+      final HRegionInfo hri, final ServerName sn) {
+    String encodedName = hri.getEncodedName();
+    AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
+    if (failedOpenCount == null) {
+      failedOpenCount = new AtomicInteger();
+      // No need to use putIfAbsent, or extra synchronization since
+      // this whole handleRegion block is locked on the encoded region
+      // name, and failedOpenTracker is updated only in this block
+      failedOpenTracker.put(encodedName, failedOpenCount);
+    }
+    if (failedOpenCount.incrementAndGet() >= maximumAttempts) {
+      regionStates.updateRegionState(hri, State.FAILED_OPEN);
+      // remove the tracking info to save memory, also reset
+      // the count for next open initiative
+      failedOpenTracker.remove(encodedName);
+    } else {
+      // Handle this the same as if it were opened and then closed.
+      RegionState regionState = regionStates.updateRegionState(hri, State.CLOSED);
+      if (regionState != null) {
+        // When there are more than one region server a new RS is selected as the
+        // destination and the same is updated in the region plan. (HBASE-5546)
+        Set<TableName> disablingOrDisabled = null;
+        try {
+          disablingOrDisabled = ZKTable.getDisablingTables(watcher);
+          disablingOrDisabled.addAll(ZKTable.getDisabledTables(watcher));
+        } catch (KeeperException e) {
+          server.abort("Cannot retrieve info about disabling or disabled tables ", e);
+        }
+        if (disablingOrDisabled.contains(hri.getTable())) {
+          offlineDisabledRegion(hri);
+          return;
+        }
+        // ZK Node is in CLOSED state, assign it.
+         regionStates.updateRegionState(hri, RegionState.State.CLOSED);
+        // This below has to do w/ online enable/disable of a table
+        removeClosedRegion(hri);
+        try {
+          getRegionPlan(hri, sn, true);
+        } catch (HBaseIOException e) {
+          LOG.warn("Failed to get region plan", e);
+        }
+        invokeAssign(hri, false);
+      }
+    }
+  }
+
+  private void onRegionOpen(
+      final HRegionInfo hri, final ServerName sn, long openSeqNum) {
+    regionOnline(hri, sn, openSeqNum);
+    if (useZKForAssignment) {
+      try {
+        // Delete the ZNode if exists
+        ZKAssign.deleteNodeFailSilent(watcher, hri);
+      } catch (KeeperException ke) {
+        server.abort("Unexpected ZK exception deleting node " + hri, ke);
+      }
+    }
+
+    // reset the count, if any
+    failedOpenTracker.remove(hri.getEncodedName());
+    if (isTableDisabledOrDisabling(hri.getTable())) {
+      invokeUnAssign(hri);
+    }
+  }
+
+  private void onRegionClosed(final HRegionInfo hri) {
+    if (isTableDisabledOrDisabling(hri.getTable())) {
+      offlineDisabledRegion(hri);
+      return;
+    }
+    regionStates.updateRegionState(hri, RegionState.State.CLOSED);
+    // This below has to do w/ online enable/disable of a table
+    removeClosedRegion(hri);
+    invokeAssign(hri, false);
+  }
+
+  private String onRegionSplit(ServerName sn, TransitionCode code,
+      HRegionInfo p, HRegionInfo a, HRegionInfo b) {
+    RegionState rs_p = regionStates.getRegionState(p);
+    RegionState rs_a = regionStates.getRegionState(a);
+    RegionState rs_b = regionStates.getRegionState(b);
+    if (!(rs_p.isOpenOrSplittingOnServer(sn)
+        && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
+        && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
+      return "Not in state good for split";
+    }
+
+    regionStates.updateRegionState(a, State.SPLITTING_NEW, sn);
+    regionStates.updateRegionState(b, State.SPLITTING_NEW, sn);
+    regionStates.updateRegionState(p, State.SPLITTING);
+
+    if (code == TransitionCode.SPLIT) {
+      if (TEST_SKIP_SPLIT_HANDLING) {
+        return "Skipping split message, TEST_SKIP_SPLIT_HANDLING is set";
+      }
+      regionOffline(p, State.SPLIT);
+      regionOnline(a, sn, 1);
+      regionOnline(b, sn, 1);
+
+      // User could disable the table before master knows the new region.
+      if (isTableDisabledOrDisabling(p.getTable())) {
+        invokeUnAssign(a);
+        invokeUnAssign(b);
+      }
+    } else if (code == TransitionCode.SPLIT_PONR) {
+      try {
+        regionStateStore.splitRegion(p, a, b, sn);
+      } catch (IOException ioe) {
+        LOG.info("Failed to record split region " + p.getShortNameToLog());
+        return "Failed to record the splitting in meta";
+      }
+    } else if (code == TransitionCode.SPLIT_REVERTED) {
+      regionOnline(p, sn);
+      regionOffline(a);
+      regionOffline(b);
+
+      if (isTableDisabledOrDisabling(p.getTable())) {
+        invokeUnAssign(p);
+      }
+    }
+    return null;
+  }
+
+  private boolean isTableDisabledOrDisabling(TableName t) {
+    Set<TableName> disablingOrDisabled = null;
+    try {
+      disablingOrDisabled = ZKTable.getDisablingTables(watcher);
+      disablingOrDisabled.addAll(ZKTable.getDisabledTables(watcher));
+    } catch (KeeperException e) {
+      server.abort("Cannot retrieve info about disabling or disabled tables ", e);
+    }
+    return disablingOrDisabled.contains(t) ? true : false;
+  }
+
+  private String onRegionMerge(ServerName sn, TransitionCode code,
+      HRegionInfo p, HRegionInfo a, HRegionInfo b) {
+    RegionState rs_p = regionStates.getRegionState(p);
+    RegionState rs_a = regionStates.getRegionState(a);
+    RegionState rs_b = regionStates.getRegionState(b);
+    if (!(rs_a.isOpenOrMergingOnServer(sn) && rs_b.isOpenOrMergingOnServer(sn)
+        && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
+      return "Not in state good for merge";
+    }
+
+    regionStates.updateRegionState(a, State.MERGING);
+    regionStates.updateRegionState(b, State.MERGING);
+    regionStates.updateRegionState(p, State.MERGING_NEW, sn);
+
+    String encodedName = p.getEncodedName();
+    if (code == TransitionCode.READY_TO_MERGE) {
+      mergingRegions.put(encodedName,
+        new PairOfSameType<HRegionInfo>(a, b));
+    } else if (code == TransitionCode.MERGED) {
+      mergingRegions.remove(encodedName);
+      regionOffline(a, State.MERGED);
+      regionOffline(b, State.MERGED);
+      regionOnline(p, sn, 1);
+
+      // User could disable the table before master knows the new region.
+      if (isTableDisabledOrDisabling(p.getTable())) {
+        invokeUnAssign(p);
+      }
+    } else if (code == TransitionCode.MERGE_PONR) {
+      try {
+        regionStateStore.mergeRegions(p, a, b, sn);
+      } catch (IOException ioe) {
+        LOG.info("Failed to record merged region " + p.getShortNameToLog());
+        return "Failed to record the merging in meta";
+      }
+    } else {
+      mergingRegions.remove(encodedName);
+      regionOnline(a, sn);
+      regionOnline(b, sn);
+      regionOffline(p);
+
+      if (isTableDisabledOrDisabling(p.getTable())) {
+        invokeUnAssign(a);
+        invokeUnAssign(b);
+      }
+    }
+    return null;
+  }
+
   /**
    * A helper to handle region merging transition event.
    * It transitions merging regions to MERGING state.
@@ -3399,7 +3813,6 @@ public class AssignmentManager extends ZooKeeperListener {
       regionStates.updateRegionState(p, State.MERGING_NEW, sn);
 
       if (et != EventType.RS_ZK_REGION_MERGED) {
-        regionStates.regionOffline(p, State.MERGING_NEW);
         this.mergingRegions.put(encodedName,
           new PairOfSameType<HRegionInfo>(hri_a, hri_b));
       } else {
@@ -3518,8 +3931,6 @@ public class AssignmentManager extends ZooKeeperListener {
     synchronized (regionStates) {
       regionStates.updateRegionState(hri_a, State.SPLITTING_NEW, sn);
       regionStates.updateRegionState(hri_b, State.SPLITTING_NEW, sn);
-      regionStates.regionOffline(hri_a, State.SPLITTING_NEW);
-      regionStates.regionOffline(hri_b, State.SPLITTING_NEW);
       regionStates.updateRegionState(rt, State.SPLITTING);
 
       // The below is for testing ONLY!  We can't do fault injection easily, so
@@ -3602,6 +4013,121 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
+   * Try to update some region states. If the state machine prevents
+   * such update, an error message is returned to explain the reason.
+   *
+   * It's expected that in each transition there should have just one
+   * region for opening/closing, 3 regions for splitting/merging.
+   * These regions should be on the server that requested the change.
+   *
+   * Region state machine. Only these transitions
+   * are expected to be triggered by a region server.
+   *
+   * On the state transition:
+   *  (1) Open/Close should be initiated by master
+   *      (a) Master sets the region to pending_open/pending_close
+   *        in memory and hbase:meta after sending the request
+   *        to the region server
+   *      (b) Region server reports back to the master
+   *        after open/close is done (either success/failure)
+   *      (c) If region server has problem to report the status
+   *        to master, it must be because the master is down or some
+   *        temporary network issue. Otherwise, the region server should
+   *        abort since it must be a bug. If the master is not accessible,
+   *        the region server should keep trying until the server is
+   *        stopped or till the status is reported to the (new) master
+   *      (d) If region server dies in the middle of opening/closing
+   *        a region, SSH picks it up and finishes it
+   *      (e) If master dies in the middle, the new master recovers
+   *        the state during initialization from hbase:meta. Region server
+   *        can report any transition that has not been reported to
+   *        the previous active master yet
+   *  (2) Split/merge is initiated by region servers
+   *      (a) To split a region, a region server sends a request
+   *        to master to try to set a region to splitting, together with
+   *        two daughters (to be created) to splitting new. If approved
+   *        by the master, the splitting can then move ahead
+   *      (b) To merge two regions, a region server sends a request to
+   *        master to try to set the new merged region (to be created) to
+   *        merging_new, together with two regions (to be merged) to merging.
+   *        If it is ok with the master, the merge can then move ahead
+   *      (c) Once the splitting/merging is done, the region server
+   *        reports the status back to the master either success/failure.
+   *      (d) Other scenarios should be handled similarly as for
+   *        region open/close
+   */
+  protected String onRegionTransition(final ServerName serverName,
+      final RegionServerStatusProtos.RegionTransition transition) {
+    TransitionCode code = transition.getTransitionCode();
+    HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
+    RegionState current = regionStates.getRegionState(hri);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Got transition " + code + " for "
+        + (current != null ? current.toString() : hri.getShortNameToLog())
+        + " from " + serverName);
+    }
+    String errorMsg = null;
+    switch (code) {
+    case OPENED:
+    case FAILED_OPEN:
+      if (current == null
+          || !current.isPendingOpenOrOpeningOnServer(serverName)) {
+        errorMsg = hri.getShortNameToLog()
+          + " is not pending open on " + serverName;
+      } else if (code == TransitionCode.FAILED_OPEN) {
+        onRegionFailedOpen(hri, serverName);
+      } else {
+        long openSeqNum = HConstants.NO_SEQNUM;
+        if (transition.hasOpenSeqNum()) {
+          openSeqNum = transition.getOpenSeqNum();
+        }
+        if (openSeqNum < 0) {
+          errorMsg = "Newly opened region has invalid open seq num " + openSeqNum;
+        } else {
+          onRegionOpen(hri, serverName, openSeqNum);
+        }
+      }
+      break;
+
+    case CLOSED:
+      if (current == null
+          || !current.isPendingCloseOrClosingOnServer(serverName)) {
+        errorMsg = hri.getShortNameToLog()
+          + " is not pending close on " + serverName;
+      } else {
+        onRegionClosed(hri);
+      }
+      break;
+
+    case READY_TO_SPLIT:
+    case SPLIT_PONR:
+    case SPLIT:
+    case SPLIT_REVERTED:
+      errorMsg = onRegionSplit(serverName, code, hri,
+        HRegionInfo.convert(transition.getRegionInfo(1)),
+        HRegionInfo.convert(transition.getRegionInfo(2)));
+      break;
+
+    case READY_TO_MERGE:
+    case MERGE_PONR:
+    case MERGED:
+    case MERGE_REVERTED:
+      errorMsg = onRegionMerge(serverName, code, hri,
+        HRegionInfo.convert(transition.getRegionInfo(1)),
+        HRegionInfo.convert(transition.getRegionInfo(2)));
+      break;
+
+    default:
+      errorMsg = "Unexpected transition code " + code;
+    }
+    if (errorMsg != null) {
+      LOG.error("Failed to transtion region from " + current + " to "
+        + code + " by " + serverName + ": " + errorMsg);
+    }
+    return errorMsg;
+  }
+
+  /**
    * @return Instance of load balancer
    */
   public LoadBalancer getBalancer() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/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 15d6ee3..62b1c6b 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
@@ -203,8 +203,11 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -3207,4 +3210,30 @@ MasterServices, Server {
     return tableNames;
   }
 
+  @Override
+  public ReportRegionTransitionResponse reportRegionTransition(RpcController controller,
+      ReportRegionTransitionRequest req) throws ServiceException {
+    try {
+      RegionTransition rt = req.getTransition(0);
+      TableName tableName = ProtobufUtil.toTableName(
+        rt.getRegionInfo(0).getTableName());
+      if (!TableName.META_TABLE_NAME.equals(tableName)
+          && !assignmentManager.isFailoverCleanupDone()) {
+        // Meta region is assigned before master finishes the
+        // failover cleanup. So no need this check for it
+        throw new PleaseHoldException("Master is rebuilding user regions");
+      }
+      ServerName sn = ProtobufUtil.toServerName(req.getServer());
+      String error = assignmentManager.onRegionTransition(sn, rt);
+      ReportRegionTransitionResponse.Builder rrtr =
+        ReportRegionTransitionResponse.newBuilder();
+      if (error != null) {
+        rrtr.setErrorMessage(error);
+      }
+      return rrtr.build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
new file mode 100644
index 0000000..d9f2262
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConfigUtil;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A helper to persist region state in meta. We may change this class
+ * to StateStore later if we also use it to store other states in meta
+ */
+@InterfaceAudience.Private
+public class RegionStateStore {
+  private static final Log LOG = LogFactory.getLog(RegionStateStore.class);
+
+  private volatile HRegion metaRegion;
+  private volatile HTableInterface metaTable;
+  private volatile boolean initialized;
+
+  private final boolean noPersistence;
+  private final CatalogTracker catalogTracker;
+  private final Server server;
+
+  /**
+   * Returns the {@link ServerName} from catalog table {@link Result}
+   * where the region is transitioning. It should be the same as
+   * {@link HRegionInfo#getServerName(Result)} if the server is at OPEN state.
+   * @param r Result to pull from
+   * @return A ServerName instance or null if necessary fields not found or empty.
+   */
+  static ServerName getRegionServer(final Result r) {
+    Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, HConstants.SERVERNAME_QUALIFIER);
+    if (cell == null || cell.getValueLength() == 0) return HRegionInfo.getServerName(r);
+    return ServerName.parseServerName(Bytes.toString(cell.getValueArray(),
+      cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  /**
+   * Pull the region state from a catalog table {@link Result}.
+   * @param r Result to pull the region state from
+   * @return the region state, or OPEN if there's no value written.
+   */
+  static State getRegionState(final Result r) {
+    Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER);
+    if (cell == null || cell.getValueLength() == 0) return State.OPEN;
+    return State.valueOf(Bytes.toString(cell.getValueArray(),
+      cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  /**
+   * Check if we should persist a state change in meta. Generally it's
+   * better to persist all state changes. However, we should not do that
+   * if the region is not in meta at all. Based on the state and the
+   * previous state, we can identify if a user region has an entry
+   * in meta. For example, merged regions are deleted from meta;
+   * New merging parents, or splitting daughters are
+   * not created in meta yet.
+   */
+  private boolean shouldPersistStateChange(
+      HRegionInfo hri, RegionState state, RegionState oldState) {
+    return !hri.isMetaRegion() && !RegionStates.isOneOfStates(
+      state, State.MERGING_NEW, State.SPLITTING_NEW, State.MERGED)
+      && !(RegionStates.isOneOfStates(state, State.OFFLINE)
+        && RegionStates.isOneOfStates(oldState, State.MERGING_NEW,
+          State.SPLITTING_NEW, State.MERGED));
+  }
+
+  RegionStateStore(final Server server) {
+    Configuration conf = server.getConfiguration();
+    // No need to persist if using ZK but not migrating
+    noPersistence = ConfigUtil.useZKForAssignment(conf)
+      && !conf.getBoolean("hbase.assignment.usezk.migrating", false);
+    catalogTracker = server.getCatalogTracker();
+    this.server = server;
+    initialized = false;
+  }
+
+  @SuppressWarnings("deprecation")
+  void start() throws IOException {
+    if (!noPersistence) {
+      if (server instanceof RegionServerServices) {
+        metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
+          HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+      }
+      if (metaRegion == null) {
+        metaTable = new HTable(TableName.META_TABLE_NAME,
+          catalogTracker.getConnection());
+      }
+    }
+    initialized = true;
+  }
+
+  void stop() {
+    initialized = false;
+    if (metaTable != null) {
+      try {
+        metaTable.close();
+      } catch (IOException e) {
+        LOG.info("Got exception in closing meta table", e);
+      } finally {
+        metaTable = null;
+      }
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  void updateRegionState(long openSeqNum,
+      RegionState newState, RegionState oldState) {
+    if (noPersistence || !initialized) {
+      return;
+    }
+
+    HRegionInfo hri = newState.getRegion();
+    if (!shouldPersistStateChange(hri, newState, oldState)) {
+      return;
+    }
+
+    ServerName oldServer = oldState != null ? oldState.getServerName() : null;
+    ServerName serverName = newState.getServerName();
+    State state = newState.getState();
+
+    try {
+      Put put = new Put(hri.getRegionName());
+      StringBuilder info = new StringBuilder("Updating row ");
+      info.append(hri.getRegionNameAsString()).append(" with state=").append(state);
+      if (serverName != null && !serverName.equals(oldServer)) {
+        put.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVERNAME_QUALIFIER,
+          Bytes.toBytes(serverName.getServerName()));
+        info.append("&sn=").append(serverName);
+      }
+      if (openSeqNum >= 0) {
+        Preconditions.checkArgument(state == State.OPEN
+          && serverName != null, "Open region should be on a server");
+        put.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
+          Bytes.toBytes(serverName.getHostAndPort()));
+        put.addImmutable(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
+          Bytes.toBytes(serverName.getStartcode()));
+        put.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER,
+          Bytes.toBytes(openSeqNum));
+        info.append("&openSeqNum=").append(openSeqNum);
+        info.append("&server=").append(serverName);
+      }
+      put.addImmutable(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
+        Bytes.toBytes(state.name()));
+      LOG.info(info);
+
+      // Persist the state change to meta
+      if (metaRegion != null) {
+        try {
+          // Assume meta is pinned to master.
+          // At least, that's what we want.
+          metaRegion.put(put);
+          return; // Done here
+        } catch (Throwable t) {
+          // In unit tests, meta could be moved away by intention
+          // So, the shortcut is gone. We won't try to establish the
+          // shortcut any more because we prefer meta to be pinned
+          // to the master
+          synchronized (this) {
+            if (metaRegion != null) {
+              LOG.info("Meta region shortcut failed", t);
+              metaTable = new HTable(TableName.META_TABLE_NAME,
+                catalogTracker.getConnection());
+              metaRegion = null;
+            }
+          }
+        }
+      }
+      synchronized(metaTable) {
+        metaTable.put(put);
+      }
+    } catch (IOException ioe) {
+      LOG.error("Failed to persist region state " + newState, ioe);
+      server.abort("Failed to update region location", ioe);
+    }
+  }
+
+  void splitRegion(HRegionInfo p,
+      HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
+    MetaEditor.splitRegion(catalogTracker, p, a, b, sn);
+  }
+
+  void mergeRegions(HRegionInfo p,
+      HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
+    MetaEditor.mergeRegions(catalogTracker, p, a, b, sn);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
index e50d737..055f370 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.Server;
@@ -67,7 +68,7 @@ public class RegionStates {
    * Region encoded name to state map.
    * All the regions should be in this map.
    */
-  private final Map<String, RegionState> regionStates;
+  private final HashMap<String, RegionState> regionStates;
 
   /**
    * Server to regions assignment map.
@@ -110,6 +111,7 @@ public class RegionStates {
   private final HashMap<ServerName, Long> processedServers;
   private long lastProcessedServerCleanTime;
 
+  private final RegionStateStore regionStateStore;
   private final ServerManager serverManager;
   private final Server server;
 
@@ -117,7 +119,8 @@ public class RegionStates {
   static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime";
   static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours
 
-  RegionStates(final Server master, final ServerManager serverManager) {
+  RegionStates(final Server master,
+      final ServerManager serverManager, final RegionStateStore regionStateStore) {
     regionStates = new HashMap<String, RegionState>();
     regionsInTransition = new HashMap<String, RegionState>();
     serverHoldings = new HashMap<ServerName, Set<HRegionInfo>>();
@@ -125,6 +128,7 @@ public class RegionStates {
     lastAssignments = new HashMap<String, ServerName>();
     processedServers = new HashMap<ServerName, Long>();
     deadServers = new HashMap<String, Long>();
+    this.regionStateStore = regionStateStore;
     this.serverManager = serverManager;
     this.server = master;
   }
@@ -188,7 +192,7 @@ public class RegionStates {
   /**
    * @return True if specified region is in one of the specified states.
    */
-  public synchronized boolean isRegionInState(
+  public boolean isRegionInState(
       final HRegionInfo hri, final State... states) {
     return isRegionInState(hri.getEncodedName(), states);
   }
@@ -196,14 +200,10 @@ public class RegionStates {
   /**
    * @return True if specified region is in one of the specified states.
    */
-  public synchronized boolean isRegionInState(
+  public boolean isRegionInState(
       final String encodedName, final State... states) {
     RegionState regionState = getRegionState(encodedName);
-    State s = regionState != null ? regionState.getState() : null;
-    for (State state: states) {
-      if (s == state) return true;
-    }
-    return false;
+    return isOneOfStates(regionState, states);
   }
 
   /**
@@ -217,9 +217,8 @@ public class RegionStates {
   /**
    * Get region transition state
    */
-  public synchronized RegionState
-      getRegionTransitionState(final HRegionInfo hri) {
-    return regionsInTransition.get(hri.getEncodedName());
+  public RegionState getRegionTransitionState(final HRegionInfo hri) {
+    return getRegionTransitionState(hri.getEncodedName());
   }
 
   /**
@@ -235,7 +234,7 @@ public class RegionStates {
    * and offline, its state will be SPLIT. Otherwise, its state will
    * be OFFLINE. Region already in RegionStates will be skipped.
    */
-  public synchronized void createRegionStates(
+  public void createRegionStates(
       final List<HRegionInfo> hris) {
     for (HRegionInfo hri: hris) {
       createRegionState(hri);
@@ -248,16 +247,44 @@ public class RegionStates {
    * be OFFLINE. If it is already in RegionStates, this call has
    * no effect, and the original state is returned.
    */
-  public synchronized RegionState createRegionState(final HRegionInfo hri) {
-    State newState = (hri.isOffline() && hri.isSplit()) ? State.SPLIT : State.OFFLINE;
+  public RegionState createRegionState(final HRegionInfo hri) {
+    return createRegionState(hri, null, null);
+  }
+
+  /**
+   * Add a region to RegionStates with the specified state.
+   * If the region is already in RegionStates, this call has
+   * no effect, and the original state is returned.
+   */
+  public synchronized RegionState createRegionState(
+      final HRegionInfo hri, State newState, ServerName serverName) {
+    if (newState == null || (newState == State.OPEN && serverName == null)) {
+      newState =  State.OFFLINE;
+    }
+    if (hri.isOffline() && hri.isSplit()) {
+      newState = State.SPLIT;
+      serverName = null;
+    }
     String encodedName = hri.getEncodedName();
     RegionState regionState = regionStates.get(encodedName);
     if (regionState != null) {
       LOG.warn("Tried to create a state for a region already in RegionStates, "
         + "used existing: " + regionState + ", ignored new: " + newState);
     } else {
-      regionState = new RegionState(hri, newState);
+      regionState = new RegionState(hri, newState, serverName);
       regionStates.put(encodedName, regionState);
+      if (newState == State.OPEN) {
+        regionAssignments.put(hri, serverName);
+        lastAssignments.put(encodedName, serverName);
+        Set<HRegionInfo> regions = serverHoldings.get(serverName);
+        if (regions == null) {
+          regions = new HashSet<HRegionInfo>();
+          serverHoldings.put(serverName, regions);
+        }
+        regions.add(hri);
+      } else if (!regionState.isUnassignable()) {
+        regionsInTransition.put(encodedName, regionState);
+      }
     }
     return regionState;
   }
@@ -265,9 +292,9 @@ public class RegionStates {
   /**
    * Update a region state. It will be put in transition if not already there.
    */
-  public synchronized RegionState updateRegionState(
+  public RegionState updateRegionState(
       final HRegionInfo hri, final State state) {
-    RegionState regionState = regionStates.get(hri.getEncodedName());
+    RegionState regionState = getRegionState(hri.getEncodedName());
     return updateRegionState(hri, state,
       regionState == null ? null : regionState.getServerName());
   }
@@ -278,7 +305,7 @@ public class RegionStates {
    * If we can't find the region info based on the region name in
    * the transition, log a warning and return null.
    */
-  public synchronized RegionState updateRegionState(
+  public RegionState updateRegionState(
       final RegionTransition transition, final State state) {
     byte [] regionName = transition.getRegionName();
     HRegionInfo regionInfo = getRegionInfo(regionName);
@@ -297,54 +324,14 @@ public class RegionStates {
   /**
    * Update a region state. It will be put in transition if not already there.
    */
-  public synchronized RegionState updateRegionState(
+  public RegionState updateRegionState(
       final HRegionInfo hri, final State state, final ServerName serverName) {
-    if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
-      LOG.warn("Failed to open/close " + hri.getShortNameToLog()
-        + " on " + serverName + ", set to " + state);
-    }
-
-    String encodedName = hri.getEncodedName();
-    RegionState regionState = new RegionState(
-      hri, state, System.currentTimeMillis(), serverName);
-    regionsInTransition.put(encodedName, regionState);
-    RegionState oldState = regionStates.put(encodedName, regionState);
-    ServerName oldServerName = oldState == null ? null : oldState.getServerName();
-    if (oldState == null || oldState.getState() != regionState.getState()
-        || (oldServerName == null && serverName != null)
-        || (oldServerName != null && !oldServerName.equals(serverName))) {
-      LOG.info("Transitioned " + oldState + " to " + regionState);
-    }
-
-    // For these states, region should be properly closed.
-    // There should be no log splitting issue.
-    if ((state == State.CLOSED || state == State.MERGED
-        || state == State.SPLIT) && lastAssignments.containsKey(encodedName)) {
-      ServerName last = lastAssignments.get(encodedName);
-      if (last.equals(serverName)) {
-        lastAssignments.remove(encodedName);
-      } else {
-        LOG.warn(encodedName + " moved to " + state + " on "
-          + serverName + ", expected " + last);
-      }
-    }
-
-    // Once a region is opened, record its last assignment right away.
-    if (serverName != null && state == State.OPEN) {
-      ServerName last = lastAssignments.get(encodedName);
-      if (!serverName.equals(last)) {
-        lastAssignments.put(encodedName, serverName);
-        if (last != null && isServerDeadAndNotProcessed(last)) {
-          LOG.warn(encodedName + " moved to " + serverName
-            + ", while it's previous host " + last
-            + " is dead but not processed yet");
-        }
-      }
-    }
+    return updateRegionState(hri, state, serverName, HConstants.NO_SEQNUM);
+  }
 
-    // notify the change
-    this.notifyAll();
-    return regionState;
+  public void regionOnline(
+      final HRegionInfo hri, final ServerName serverName) {
+    regionOnline(hri, serverName, HConstants.NO_SEQNUM);
   }
 
   /**
@@ -352,8 +339,8 @@ public class RegionStates {
    * We can't confirm it is really online on specified region server
    * because it hasn't been put in region server's online region list yet.
    */
-  public synchronized void regionOnline(
-      final HRegionInfo hri, final ServerName serverName) {
+  public void regionOnline(final HRegionInfo hri,
+      final ServerName serverName, long openSeqNum) {
     if (!serverManager.isServerOnline(serverName)) {
       // This is possible if the region server dies before master gets a
       // chance to handle ZK event in time. At this time, if the dead server
@@ -363,30 +350,26 @@ public class RegionStates {
         + " was opened on a dead server: " + serverName);
       return;
     }
+    updateRegionState(hri, State.OPEN, serverName, openSeqNum);
 
-    String encodedName = hri.getEncodedName();
-    RegionState oldState = regionStates.get(encodedName);
-    if (oldState == null) {
-      LOG.warn("Online region not in RegionStates: " + hri.getShortNameToLog());
-    }
-    updateRegionState(hri, State.OPEN, serverName);
-    regionsInTransition.remove(encodedName);
-
-    ServerName oldServerName = regionAssignments.put(hri, serverName);
-    if (!serverName.equals(oldServerName)) {
-      LOG.info("Onlined " + hri.getShortNameToLog() + " on " + serverName);
-      Set<HRegionInfo> regions = serverHoldings.get(serverName);
-      if (regions == null) {
-        regions = new HashSet<HRegionInfo>();
-        serverHoldings.put(serverName, regions);
-      }
-      regions.add(hri);
-      if (oldServerName != null) {
-        LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
-        Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
-        oldRegions.remove(hri);
-        if (oldRegions.isEmpty()) {
-          serverHoldings.remove(oldServerName);
+    synchronized (this) {
+      regionsInTransition.remove(hri.getEncodedName());
+      ServerName oldServerName = regionAssignments.put(hri, serverName);
+      if (!serverName.equals(oldServerName)) {
+        LOG.info("Onlined " + hri.getShortNameToLog() + " on " + serverName);
+        Set<HRegionInfo> regions = serverHoldings.get(serverName);
+        if (regions == null) {
+          regions = new HashSet<HRegionInfo>();
+          serverHoldings.put(serverName, regions);
+        }
+        regions.add(hri);
+        if (oldServerName != null) {
+          LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
+          Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
+          oldRegions.remove(hri);
+          if (oldRegions.isEmpty()) {
+            serverHoldings.remove(oldServerName);
+          }
         }
       }
     }
@@ -405,6 +388,9 @@ public class RegionStates {
       }
     }
     long now = System.currentTimeMillis();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Adding to processed servers " + serverName);
+    }
     processedServers.put(serverName, Long.valueOf(now));
     Configuration conf = server.getConfiguration();
     long obsoleteTime = conf.getLong(LOG_SPLIT_TIME, DEFAULT_LOG_SPLIT_TIME);
@@ -416,6 +402,9 @@ public class RegionStates {
           = processedServers.entrySet().iterator(); it.hasNext();) {
         Map.Entry<ServerName, Long> e = it.next();
         if (e.getValue().longValue() < cutoff) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Removed from processed servers " + e.getKey());
+          }
           it.remove();
         }
       }
@@ -425,7 +414,7 @@ public class RegionStates {
   /**
    * Log split is done for a given region, so it is assignable now.
    */
-  public synchronized void logSplit(final HRegionInfo region) {
+  public void logSplit(final HRegionInfo region) {
     clearLastAssignment(region);
   }
 
@@ -445,24 +434,30 @@ public class RegionStates {
    * should be the specified expected state, which can only be
    * Split/Merged/Offline/null(=Offline)/SplittingNew/MergingNew.
    */
-  public synchronized void regionOffline(
+  public void regionOffline(
       final HRegionInfo hri, final State expectedState) {
     Preconditions.checkArgument(expectedState == null
       || RegionState.isUnassignable(expectedState),
         "Offlined region should not be " + expectedState);
-    String encodedName = hri.getEncodedName();
+    if (isRegionInState(hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
+      // Remove it from all region maps
+      deleteRegion(hri);
+      return;
+    }
     State newState =
       expectedState == null ? State.OFFLINE : expectedState;
     updateRegionState(hri, newState);
-    regionsInTransition.remove(encodedName);
 
-    ServerName oldServerName = regionAssignments.remove(hri);
-    if (oldServerName != null) {
-      LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
-      Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
-      oldRegions.remove(hri);
-      if (oldRegions.isEmpty()) {
-        serverHoldings.remove(oldServerName);
+    synchronized (this) {
+      regionsInTransition.remove(hri.getEncodedName());
+      ServerName oldServerName = regionAssignments.remove(hri);
+      if (oldServerName != null && serverHoldings.containsKey(oldServerName)) {
+        LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
+        Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
+        oldRegions.remove(hri);
+        if (oldRegions.isEmpty()) {
+          serverHoldings.remove(oldServerName);
+        }
       }
     }
   }
@@ -580,12 +575,14 @@ public class RegionStates {
    * A table is deleted. Remove its regions from all internal maps.
    * We loop through all regions assuming we don't delete tables too much.
    */
-  public synchronized void tableDeleted(final TableName tableName) {
+  public void tableDeleted(final TableName tableName) {
     Set<HRegionInfo> regionsToDelete = new HashSet<HRegionInfo>();
-    for (RegionState state: regionStates.values()) {
-      HRegionInfo region = state.getRegion();
-      if (region.getTable().equals(tableName)) {
-        regionsToDelete.add(region);
+    synchronized (this) {
+      for (RegionState state: regionStates.values()) {
+        HRegionInfo region = state.getRegion();
+        if (region.getTable().equals(tableName)) {
+          regionsToDelete.add(region);
+        }
       }
     }
     for (HRegionInfo region: regionsToDelete) {
@@ -654,6 +651,21 @@ public class RegionStates {
     lastAssignments.put(encodedName, serverName);
   }
 
+  synchronized void closeAllUserRegions(Set<TableName> excludedTables) {
+    Set<HRegionInfo> toBeClosed = new HashSet<HRegionInfo>(regionStates.size());
+    for(RegionState state: regionStates.values()) {
+      HRegionInfo hri = state.getRegion();
+      TableName tableName = hri.getTable();
+      if (!hri.isSplit() && !hri.isMetaRegion()
+          && !excludedTables.contains(tableName)) {
+        toBeClosed.add(hri);
+      }
+    }
+    for (HRegionInfo hri: toBeClosed) {
+      updateRegionState(hri, State.CLOSED);
+    }
+  }
+
   /**
    * Compute the average load across all region servers.
    * Currently, this uses a very naive computation - just uses the number of
@@ -729,8 +741,8 @@ public class RegionStates {
     return result;
   }
 
-  protected synchronized RegionState getRegionState(final HRegionInfo hri) {
-    return regionStates.get(hri.getEncodedName());
+  protected RegionState getRegionState(final HRegionInfo hri) {
+    return getRegionState(hri.getEncodedName());
   }
 
   protected synchronized RegionState getRegionState(final String encodedName) {
@@ -744,7 +756,7 @@ public class RegionStates {
    */
   protected HRegionInfo getRegionInfo(final byte [] regionName) {
     String encodedName = HRegionInfo.encodeRegionName(regionName);
-    RegionState regionState = regionStates.get(encodedName);
+    RegionState regionState = getRegionState(encodedName);
     if (regionState != null) {
       return regionState.getRegion();
     }
@@ -764,10 +776,74 @@ public class RegionStates {
     }
   }
 
+  static boolean isOneOfStates(RegionState regionState, State... states) {
+    State s = regionState != null ? regionState.getState() : null;
+    for (State state: states) {
+      if (s == state) return true;
+    }
+    return false;
+  }
+
+  /**
+   * Update a region state. It will be put in transition if not already there.
+   */
+  private RegionState updateRegionState(final HRegionInfo hri,
+      final State state, final ServerName serverName, long openSeqNum) {
+    if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
+      LOG.warn("Failed to open/close " + hri.getShortNameToLog()
+        + " on " + serverName + ", set to " + state);
+    }
+
+    String encodedName = hri.getEncodedName();
+    RegionState regionState = new RegionState(
+      hri, state, System.currentTimeMillis(), serverName);
+    RegionState oldState = getRegionState(encodedName);
+    if (!regionState.equals(oldState)) {
+      LOG.info("Transition " + oldState + " to " + regionState);
+      // Persist region state before updating in-memory info, if needed
+      regionStateStore.updateRegionState(openSeqNum, regionState, oldState);
+    }
+
+    synchronized (this) {
+      regionsInTransition.put(encodedName, regionState);
+      regionStates.put(encodedName, regionState);
+
+      // For these states, region should be properly closed.
+      // There should be no log splitting issue.
+      if ((state == State.CLOSED || state == State.MERGED
+          || state == State.SPLIT) && lastAssignments.containsKey(encodedName)) {
+        ServerName last = lastAssignments.get(encodedName);
+        if (last.equals(serverName)) {
+          lastAssignments.remove(encodedName);
+        } else {
+          LOG.warn(encodedName + " moved to " + state + " on "
+            + serverName + ", expected " + last);
+        }
+      }
+
+      // Once a region is opened, record its last assignment right away.
+      if (serverName != null && state == State.OPEN) {
+        ServerName last = lastAssignments.get(encodedName);
+        if (!serverName.equals(last)) {
+          lastAssignments.put(encodedName, serverName);
+          if (last != null && isServerDeadAndNotProcessed(last)) {
+            LOG.warn(encodedName + " moved to " + serverName
+              + ", while it's previous host " + last
+              + " is dead but not processed yet");
+          }
+        }
+      }
+
+      // notify the change
+      this.notifyAll();
+    }
+    return regionState;
+  }
+
   /**
    * Remove a region from all state maps.
    */
-  private void deleteRegion(final HRegionInfo hri) {
+  private synchronized void deleteRegion(final HRegionInfo hri) {
     String encodedName = hri.getEncodedName();
     regionsInTransition.remove(encodedName);
     regionStates.remove(encodedName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 4cc55dd..42f4adb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -171,7 +171,8 @@ public class ServerManager {
    * is currently in startup mode. In this case, the dead server will be parked in this set
    * temporarily.
    */
-  private Map<ServerName, Boolean> requeuedDeadServers = new HashMap<ServerName, Boolean>();
+  private Map<ServerName, Boolean> requeuedDeadServers
+    = new ConcurrentHashMap<ServerName, Boolean>();
 
   /** Listeners that are called on server events. */
   private List<ServerListener> listeners = new CopyOnWriteArrayList<ServerListener>();
@@ -912,6 +913,15 @@ public class ServerManager {
   }
 
   /**
+   * During startup, if we figure it is not a failover, i.e. there is
+   * no more HLog files to split, we won't try to recover these dead servers.
+   * So we just remove them from the queue. Use caution in calling this.
+   */
+  void removeRequeuedDeadServers() {
+    requeuedDeadServers.clear();
+  }
+
+  /**
    * @return A copy of the internal map of requeuedDeadServers servers and their corresponding
    *         splitlog need flag.
    */


[05/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

HBASE-11059 ZK-less region assignment (Jimmy Xiang)


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

Branch: refs/heads/0.98
Commit: e6ffa86e33ee173afcff15ca4b614e6ec56357ed
Parents: 5aa39bb
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:09 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:03:10 2014 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    |    1 +
 .../org/apache/hadoop/hbase/RegionLoad.java     |    4 +-
 .../apache/hadoop/hbase/catalog/MetaReader.java |   10 +-
 .../apache/hadoop/hbase/master/RegionState.java |   32 +-
 .../hbase/zookeeper/ZooKeeperWatcher.java       |    4 +-
 .../org/apache/hadoop/hbase/HConstants.java     |   10 +
 .../generated/RegionServerStatusProtos.java     | 2959 +++++++++++++++++-
 .../src/main/protobuf/RegionServerStatus.proto  |   49 +
 .../hadoop/hbase/master/AssignCallable.java     |    7 +-
 .../hadoop/hbase/master/AssignmentManager.java  |  778 ++++-
 .../org/apache/hadoop/hbase/master/HMaster.java |   29 +
 .../hadoop/hbase/master/RegionStateStore.java   |  225 ++
 .../hadoop/hbase/master/RegionStates.java       |  296 +-
 .../hadoop/hbase/master/ServerManager.java      |   12 +-
 .../master/handler/ServerShutdownHandler.java   |   27 +-
 .../hbase/regionserver/HRegionServer.java       |   97 +-
 .../regionserver/RegionMergeTransaction.java    |  111 +-
 .../regionserver/RegionServerServices.java      |   11 +
 .../hbase/regionserver/SplitTransaction.java    |   70 +-
 .../handler/CloseRegionHandler.java             |   30 +-
 .../regionserver/handler/OpenRegionHandler.java |   42 +-
 .../apache/hadoop/hbase/util/ConfigUtil.java    |   33 +
 .../hadoop/hbase/MockRegionServerServices.java  |   15 +-
 .../apache/hadoop/hbase/TestDrainingServer.java |    1 +
 .../client/TestScannersFromClientSide.java      |    5 +-
 .../hadoop/hbase/master/MockRegionServer.java   |   12 +
 .../hbase/master/TestAssignmentManager.java     |    2 +
 .../master/TestAssignmentManagerOnCluster.java  |   37 +-
 .../hadoop/hbase/master/TestMasterFailover.java |  103 +
 .../hbase/master/TestOpenedRegionHandler.java   |    2 +
 .../hadoop/hbase/master/TestRestartCluster.java |    1 +
 .../master/TestZKBasedOpenCloseRegion.java      |    1 +
 .../hbase/master/TestZKLessAMOnCluster.java     |   42 +
 .../TestEndToEndSplitTransaction.java           |    7 +-
 .../TestRegionMergeTransactionOnCluster.java    |   20 +-
 .../regionserver/TestRegionServerNoMaster.java  |    1 +
 .../TestSplitTransactionOnCluster.java          |  130 +-
 .../regionserver/TestZKLessMergeOnCluster.java  |   45 +
 .../regionserver/TestZKLessSplitOnCluster.java  |   45 +
 .../handler/TestCloseRegionHandler.java         |    1 +
 .../handler/TestOpenRegionHandler.java          |    1 +
 .../apache/hadoop/hbase/util/TestHBaseFsck.java |    6 +
 .../zookeeper/TestRecoverableZooKeeper.java     |    2 +-
 43 files changed, 4854 insertions(+), 462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 502688c..da2ac0d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -1048,6 +1048,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param r Result to pull from
    * @return A ServerName instance or null if necessary fields not found or empty.
    */
+  // TODO: ServerName should be a better place for this method?
   public static ServerName getServerName(final Result r) {
     Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
     if (cell == null || cell.getValueLength() == 0) return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
index 2fa45e4..28e6d6b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
@@ -203,6 +203,8 @@ public class RegionLoad {
     }
     sb = Strings.appendKeyValue(sb, "compactionProgressPct",
         compactionProgressPct);
+    sb = Strings.appendKeyValue(sb, "completeSequenceId",
+        this.getCompleteSequenceId());
     return sb.toString();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
index 4f15390..ec0f4a0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
@@ -160,6 +160,7 @@ public class MetaReader {
    * @return An {@link HTable} for <code>tableName</code>
    * @throws IOException
    */
+  @SuppressWarnings("deprecation")
   private static HTable getHTable(final CatalogTracker catalogTracker,
       final TableName tableName)
   throws IOException {
@@ -482,8 +483,11 @@ public class MetaReader {
       @Override
       void add(Result r) {
         if (r == null || r.isEmpty()) return;
+        if (HRegionInfo.getHRegionInfo(r) == null) return;
         ServerName sn = HRegionInfo.getServerName(r);
-        if (sn != null && sn.equals(serverName)) this.results.add(r);
+        if (sn != null && sn.equals(serverName)) {
+          this.results.add(r);
+        }
       }
     };
     fullScan(catalogTracker, v);
@@ -491,8 +495,8 @@ public class MetaReader {
     if (results != null && !results.isEmpty()) {
       // Convert results to Map keyed by HRI
       for (Result r: results) {
-        Pair<HRegionInfo, ServerName> p = HRegionInfo.getHRegionInfoAndServerName(r);
-        if (p != null && p.getFirst() != null) hris.put(p.getFirst(), r);
+        HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
+        if (hri != null) hris.put(hri, r);
       }
     }
     return hris;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index 5b07e88..e6aa279 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.Date;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -35,7 +32,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
  * So it is almost immutable.
  */
 @InterfaceAudience.Private
-public class RegionState implements org.apache.hadoop.io.Writable {
+public class RegionState {
 
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
@@ -77,6 +74,11 @@ public class RegionState implements org.apache.hadoop.io.Writable {
   }
 
   public RegionState(HRegionInfo region,
+      State state, ServerName serverName) {
+    this(region, state, System.currentTimeMillis(), serverName);
+  }
+
+  public RegionState(HRegionInfo region,
       State state, long stamp, ServerName serverName) {
     this.hri = region;
     this.state = state;
@@ -398,25 +400,11 @@ public class RegionState implements org.apache.hadoop.io.Writable {
   }
 
   /**
-   * @deprecated Writables are going away
-   */
-  @Deprecated
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    hri = new HRegionInfo();
-    hri.readFields(in);
-    state = State.valueOf(in.readUTF());
-    stamp.set(in.readLong());
-  }
-
-  /**
-   * @deprecated Writables are going away
+   * Don't count timestamp in hash code calculation
    */
-  @Deprecated
   @Override
-  public void write(DataOutput out) throws IOException {
-    hri.write(out);
-    out.writeUTF(state.name());
-    out.writeLong(stamp.get());
+  public int hashCode() {
+    return (serverName != null ? serverName.hashCode() * 11 : 0)
+      + hri.hashCode() + 5 * state.ordinal();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 2c88a3c..d32f6fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -172,7 +172,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     try {
       // Create all the necessary "directories" of znodes
       ZKUtil.createWithParents(this, baseZNode);
-      ZKUtil.createAndFailSilent(this, assignmentZNode);
+      if (conf.getBoolean("hbase.assignment.usezk", true)) {
+        ZKUtil.createAndFailSilent(this, assignmentZNode);
+      }
       ZKUtil.createAndFailSilent(this, rsZNode);
       ZKUtil.createAndFailSilent(this, drainingZNode);
       ZKUtil.createAndFailSilent(this, tableZNode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 379ae7b..cba4471 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -395,6 +395,16 @@ public final class HConstants {
   /** The open seqnum column qualifier */
   public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes("seqnumDuringOpen");
 
+  /** The state column qualifier */
+  public static final byte [] STATE_QUALIFIER = Bytes.toBytes("state");
+
+  /**
+   * The serverName column qualifier. Its the server where the region is
+   * transitioning on, while column server is the server where the region is
+   * opened on. They are the same when the region is in state OPEN.
+   */
+  public static final byte [] SERVERNAME_QUALIFIER = Bytes.toBytes("sn");
+
   /** The lower-half split region column qualifier */
   public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
 


[08/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [4/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [4/8]

HBASE-11659 Region state RPC call is not idempotent (Virag Kothari)


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

Branch: refs/heads/0.98
Commit: 44f646c832929a1b092320886726979ae3240f11
Parents: 7852df6
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:03:46 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/AssignmentManager.java  |  5 ++
 .../master/TestAssignmentManagerOnCluster.java  | 68 +++++++++++++++++++-
 2 files changed, 71 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/44f646c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 60e0afa..1700a56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -4071,6 +4071,11 @@ public class AssignmentManager extends ZooKeeperListener {
     String errorMsg = null;
     switch (code) {
     case OPENED:
+      if (current != null && current.isOpened() && current.isOnServer(serverName)) {
+        LOG.info("Region " + hri.getShortNameToLog() + " is already " + current.getState() + " on "
+            + serverName);
+        break;
+      }
     case FAILED_OPEN:
       if (current == null
           || !current.isPendingOpenOrOpeningOnServer(serverName)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/44f646c8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index eb15240..4e96e98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConfigUtil;
@@ -88,7 +90,7 @@ public class TestAssignmentManagerOnCluster {
     // Reduce the maximum attempts to speed up the test
     conf.setInt("hbase.assignment.maximum.attempts", 3);
 
-    TEST_UTIL.startMiniCluster(1, 4, null, MyMaster.class, null);
+    TEST_UTIL.startMiniCluster(1, 4, null, MyMaster.class, MyRegionServer.class);
     admin = TEST_UTIL.getHBaseAdmin();
   }
 
@@ -141,7 +143,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.deleteTable(Bytes.toBytes(table));
     }
   }
-
+  
   /**
    * This tests region assignment on a simulated restarted server
    */
@@ -826,6 +828,40 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.deleteTable(Bytes.toBytes(table));
     }
   }
+  
+  /**
+   * Test that region state transition call is idempotent
+   */
+  @Test(timeout = 60000)
+  public void testReportRegionStateTransition() throws Exception {
+    String table = "testReportRegionStateTransition";
+    try {
+      MyRegionServer.simulateRetry = true;
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      desc.addFamily(new HColumnDescriptor(FAMILY));
+      admin.createTable(desc);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      HRegionInfo hri =
+          new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+      MetaEditor.addRegionToMeta(meta, hri);
+      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+      master.assignRegion(hri);
+      AssignmentManager am = master.getAssignmentManager();
+      am.waitForAssignment(hri);
+      RegionStates regionStates = am.getRegionStates();
+      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
+      // Assert the the region is actually open on the server
+      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
+      // Closing region should just work fine
+      admin.disableTable(TableName.valueOf(table));
+      assertTrue(regionStates.isRegionOffline(hri));
+      List<HRegionInfo> regions = TEST_UTIL.getHBaseAdmin().getOnlineRegions(serverName);
+      assertTrue(!regions.contains(hri));
+    } finally {
+      MyRegionServer.simulateRetry = false;
+      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+    }
+  }
 
   static class MyLoadBalancer extends StochasticLoadBalancer {
     // For this region, if specified, always assign to nowhere
@@ -861,6 +897,34 @@ public class TestAssignmentManagerOnCluster {
       }
     }
   }
+  
+  public static class MyRegionServer extends MiniHBaseClusterRegionServer {
+    static volatile ServerName abortedServer = null;
+    static volatile boolean simulateRetry;
+
+    public MyRegionServer(Configuration conf)
+      throws IOException, KeeperException,
+        InterruptedException {
+      super(conf);
+    }
+
+    @Override
+    public boolean
+        reportRegionTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris) {
+      if (simulateRetry == true) {
+        // Simulate retry by calling the method twice
+        super.reportRegionTransition(code, openSeqNum, hris);
+        return super.reportRegionTransition(code, openSeqNum, hris);
+      }
+      return super.reportRegionTransition(code, openSeqNum, hris);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return getServerName().equals(abortedServer) || super.isAborted();
+    }
+  }
+
 
   public static class MyRegionObserver extends BaseRegionObserver {
     // If enabled, fail all preClose calls


[02/12] HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
index 8a9e9c7..ed1003d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
@@ -21,27 +21,25 @@ package org.apache.hadoop.hbase.master.handler;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
 import java.util.Set;
 import java.util.concurrent.locks.Lock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.DeadServer;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
@@ -163,14 +161,14 @@ public class ServerShutdownHandler extends EventHandler {
       // completed (zk is updated after edits to hbase:meta have gone in).  See
       // {@link SplitTransaction}.  We'd also have to be figure another way for
       // doing the below hbase:meta daughters fixup.
-      NavigableMap<HRegionInfo, Result> hris = null;
+      Set<HRegionInfo> hris = null;
       while (!this.server.isStopped()) {
         try {
           this.server.getCatalogTracker().waitForMeta();
           // Skip getting user regions if the server is stopped.
           if (!this.server.isStopped()) {
             hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
-                this.serverName);
+              this.serverName).keySet();
           }
           break;
         } catch (InterruptedException e) {
@@ -196,9 +194,8 @@ public class ServerShutdownHandler extends EventHandler {
           LOG.info("Splitting logs for " + serverName + " before assignment.");
           if (distributedLogReplay) {
             LOG.info("Mark regions in recovery before assignment.");
-            Set<ServerName> serverNames = new HashSet<ServerName>();
-            serverNames.add(serverName);
-            this.services.getMasterFileSystem().prepareLogReplay(serverNames);
+            MasterFileSystem mfs = this.services.getMasterFileSystem();
+            mfs.prepareLogReplay(serverName, hris);
           } else {
             this.services.getMasterFileSystem().splitLog(serverName);
           }
@@ -224,10 +221,9 @@ public class ServerShutdownHandler extends EventHandler {
       toAssignRegions.addAll(regionsInTransition);
 
       // Iterate regions that were on this server and assign them
-      if (hris != null) {
+      if (hris != null && !hris.isEmpty()) {
         RegionStates regionStates = am.getRegionStates();
-        for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) {
-          HRegionInfo hri = e.getKey();
+        for (HRegionInfo hri: hris) {
           if (regionsInTransition.contains(hri)) {
             continue;
           }
@@ -235,7 +231,7 @@ public class ServerShutdownHandler extends EventHandler {
           Lock lock = am.acquireRegionLock(encodedName);
           try {
             RegionState rit = regionStates.getRegionTransitionState(hri);
-            if (processDeadRegion(hri, e.getValue(), am, server.getCatalogTracker())) {
+            if (processDeadRegion(hri, am, server.getCatalogTracker())) {
               ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
               if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
                 // If this region is in transition on the dead server, it must be
@@ -261,7 +257,7 @@ public class ServerShutdownHandler extends EventHandler {
                 }
               } else if (regionStates.isRegionInState(
                   hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
-                regionStates.regionOffline(hri);
+                regionStates.updateRegionState(hri, State.OFFLINE);
               }
               toAssignRegions.add(hri);
             } else if (rit != null) {
@@ -334,13 +330,12 @@ public class ServerShutdownHandler extends EventHandler {
    * Process a dead region from a dead RS. Checks if the region is disabled or
    * disabling or if the region has a partially completed split.
    * @param hri
-   * @param result
    * @param assignmentManager
    * @param catalogTracker
    * @return Returns true if specified region should be assigned, false if not.
    * @throws IOException
    */
-  public static boolean processDeadRegion(HRegionInfo hri, Result result,
+  public static boolean processDeadRegion(HRegionInfo hri,
       AssignmentManager assignmentManager, CatalogTracker catalogTracker)
   throws IOException {
     boolean tablePresent = assignmentManager.getZKTable().isTablePresent(hri.getTable());

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/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 f681a72..f904238 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
@@ -191,7 +191,11 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
@@ -211,6 +215,7 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -378,7 +383,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
   protected final int numRegionsToReport;
 
   // Stub to do region server status calls against the master.
-  private RegionServerStatusService.BlockingInterface rssStub;
+  private volatile RegionServerStatusService.BlockingInterface rssStub;
   // RPC client. Used to make the stub above that does region server status checking.
   RpcClient rpcClient;
 
@@ -500,6 +505,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
   // Table level lock manager for locking for region operations
   private TableLockManager tableLockManager;
 
+  private final boolean useZKForAssignment;
+
+  // Used for 11059
+  private ServerName serverName;
+
   /**
    * Nonce manager. Nonces are used to make operations like increment and append idempotent
    * in the case where client doesn't receive the response from a successful operation and
@@ -597,6 +607,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
     } catch (IllegalAccessException e) {
       throw new IllegalArgumentException(e);
     }
+    
     this.rpcServer = new RpcServer(this, name, getServices(),
       /*HBaseRPCErrorHandler.class, OnlineRegions.class},*/
       initialIsa, // BindAddress is IP we got for this server.
@@ -605,9 +616,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
 
     // Set our address.
     this.isa = this.rpcServer.getListenerAddress();
-
+    
     this.rpcServer.setErrorHandler(this);
     this.startcode = System.currentTimeMillis();
+    serverName = ServerName.valueOf(isa.getHostName(), isa.getPort(), startcode);
+    useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
 
     // login the zookeeper client principal (if using security)
     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
@@ -1055,8 +1068,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
   @VisibleForTesting
   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
   throws IOException {
-    if (this.rssStub == null) {
-      // the current server is stopping.
+    RegionServerStatusService.BlockingInterface rss = rssStub;
+    if (rss == null) {
+      // the current server could be stopping.
       return;
     }
     ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
@@ -1066,18 +1080,19 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
         this.serverNameFromMasterPOV.getVersionedBytes());
       request.setServer(ProtobufUtil.toServerName(sn));
       request.setLoad(sl);
-      this.rssStub.regionServerReport(null, request.build());
+      rss.regionServerReport(null, request.build());
     } catch (ServiceException se) {
       IOException ioe = ProtobufUtil.getRemoteException(se);
       if (ioe instanceof YouAreDeadException) {
         // This will be caught and handled as a fatal error in run()
         throw ioe;
       }
+      if (rssStub == rss) {
+        rssStub = null;
+      }
       // Couldn't connect to the master, get location from zk and reconnect
       // Method blocks until new master is found or we are stopped
-      Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
-        createRegionServerStatusStub();
-      this.rssStub = p.getSecond();
+      createRegionServerStatusStub();
     }
   }
 
@@ -1799,10 +1814,16 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
     if (r.getRegionInfo().isMetaRegion()) {
       MetaRegionTracker.setMetaLocation(getZooKeeper(),
           this.serverNameFromMasterPOV);
-    } else {
+    } else if (useZKForAssignment) {
       MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
         this.serverNameFromMasterPOV, openSeqNum);
     }
+    if (!useZKForAssignment
+        && !reportRegionTransition(TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
+      throw new IOException("Failed to report opened region to master: "
+          + r.getRegionNameAsString());
+    }
+
     LOG.info("Finished post open deploy task for " + r.getRegionNameAsString());
 
   }
@@ -1941,6 +1962,49 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
     return replicationSinkHandler;
   }
 
+  @Override
+  public boolean reportRegionTransition(TransitionCode code, HRegionInfo... hris) {
+    return reportRegionTransition(code, HConstants.NO_SEQNUM, hris);
+  }
+
+  @Override
+  public boolean reportRegionTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris) {
+    ReportRegionTransitionRequest.Builder builder = ReportRegionTransitionRequest.newBuilder();
+    builder.setServer(ProtobufUtil.toServerName(serverName));
+    RegionTransition.Builder transition = builder.addTransitionBuilder();
+    transition.setTransitionCode(code);
+    if (code == TransitionCode.OPENED && openSeqNum >= 0) {
+      transition.setOpenSeqNum(openSeqNum);
+    }
+    for (HRegionInfo hri : hris) {
+      transition.addRegionInfo(HRegionInfo.convert(hri));
+    }
+    ReportRegionTransitionRequest request = builder.build();
+    while (keepLooping()) {
+      RegionServerStatusService.BlockingInterface rss = rssStub;
+      try {
+        if (rss == null) {
+          createRegionServerStatusStub();
+          continue;
+        }
+        ReportRegionTransitionResponse response = rss.reportRegionTransition(null, request);
+        if (response.hasErrorMessage()) {
+          LOG.info("Failed to transition " + hris[0] + " to " + code + ": "
+              + response.getErrorMessage());
+          return false;
+        }
+        return true;
+      } catch (ServiceException se) {
+        IOException ioe = ProtobufUtil.getRemoteException(se);
+        LOG.info("Failed to report region transition, will retry", ioe);
+        if (rssStub == rss) {
+          rssStub = null;
+        }
+      }
+    }
+    return false;
+  }
+
   /**
    * Get the current master from ZooKeeper and open the RPC connection to it.
    *
@@ -1949,8 +2013,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
    *
    * @return master + port, or null if server has been stopped
    */
-  private Pair<ServerName, RegionServerStatusService.BlockingInterface>
+  private synchronized ServerName
   createRegionServerStatusStub() {
+    if (rssStub != null) {
+      return masterAddressTracker.getMasterAddress();
+    }
     ServerName sn = null;
     long previousLogTime = 0;
     RegionServerStatusService.BlockingInterface master = null;
@@ -1997,7 +2064,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
         }
       }
     }
-    return new Pair<ServerName, RegionServerStatusService.BlockingInterface>(sn, intf);
+    rssStub = intf;
+    return sn;
   }
 
   /**
@@ -2016,12 +2084,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
    * @throws IOException
    */
   private RegionServerStartupResponse reportForDuty() throws IOException {
+    ServerName masterServerName = createRegionServerStatusStub();
+    if (masterServerName == null) return null;
     RegionServerStartupResponse result = null;
-    Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
-      createRegionServerStatusStub();
-    this.rssStub = p.getSecond();
-    ServerName masterServerName = p.getFirst();
-    if (masterServerName == null) return result;
     try {
       this.requestCount.set(0);
       LOG.info("reportForDuty to master=" + masterServerName + " with port=" + this.isa.getPort() +

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java
index 287ffa1..6593560 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java
@@ -46,8 +46,10 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.SplitTransaction.LoggingProgressable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@@ -99,6 +101,7 @@ public class RegionMergeTransaction {
   private int znodeVersion = -1;
   // We only merge adjacent regions if forcible is false
   private final boolean forcible;
+  private boolean useZKForAssignment;
 
   /**
    * Types to add to the transaction journal. Each enum is a step in the merge
@@ -239,6 +242,8 @@ public class RegionMergeTransaction {
    */
   public HRegion execute(final Server server,
       final RegionServerServices services) throws IOException {
+    useZKForAssignment = server == null ? true :
+      ConfigUtil.useZKForAssignment(server.getConfiguration());
     if (rsCoprocessorHost == null) {
       rsCoprocessorHost = server != null ? ((HRegionServer) server).getCoprocessorHost() : null;
     }
@@ -315,7 +320,7 @@ public class RegionMergeTransaction {
     // will determine whether the region is merged or not in case of failures.
     // If it is successful, master will roll-forward, if not, master will
     // rollback
-    if (!testing) {
+    if (!testing && useZKForAssignment) {
       if (metaEntries.isEmpty()) {
         MetaEditor.mergeRegions(server.getCatalogTracker(), mergedRegion.getRegionInfo(), region_a
             .getRegionInfo(), region_b.getRegionInfo(), server.getServerName());
@@ -323,6 +328,14 @@ public class RegionMergeTransaction {
         mergeRegionsAndPutMetaEntries(server.getCatalogTracker(), mergedRegion.getRegionInfo(),
           region_a.getRegionInfo(), region_b.getRegionInfo(), server.getServerName(), metaEntries);
       }
+    } else if (services != null && !useZKForAssignment) {
+      if (!services.reportRegionTransition(TransitionCode.MERGE_PONR,
+          mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
+        // Passed PONR, let SSH clean it up
+        throw new IOException("Failed to notify master that merge passed PONR: "
+          + region_a.getRegionInfo().getRegionNameAsString() + " and "
+          + region_b.getRegionInfo().getRegionNameAsString());
+      }
     }
     return mergedRegion;
   }
@@ -352,6 +365,7 @@ public class RegionMergeTransaction {
     addLocation(putOfMerged, serverName, 1);
   }
 
+  @SuppressWarnings("deprecation")
   public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
     p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
         .toBytes(sn.getHostAndPort()));
@@ -365,7 +379,7 @@ public class RegionMergeTransaction {
       boolean testing) throws IOException {
     // Set ephemeral MERGING znode up in zk. Mocked servers sometimes don't
     // have zookeeper so don't do zk stuff if server or zookeeper is null
-    if (server != null && server.getZooKeeper() != null) {
+    if (useZKAndZKIsSet(server)) {
       try {
         createNodeMerging(server.getZooKeeper(), this.mergedRegionInfo,
           server.getServerName(), region_a.getRegionInfo(), region_b.getRegionInfo());
@@ -373,9 +387,16 @@ public class RegionMergeTransaction {
         throw new IOException("Failed creating PENDING_MERGE znode on "
             + this.mergedRegionInfo.getRegionNameAsString(), e);
       }
+    } else if (services != null && !useZKForAssignment) {
+      if (!services.reportRegionTransition(TransitionCode.READY_TO_MERGE,
+          mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
+        throw new IOException("Failed to get ok from master to merge "
+          + region_a.getRegionInfo().getRegionNameAsString() + " and "
+          + region_b.getRegionInfo().getRegionNameAsString());
+      }
     }
     this.journal.add(JournalEntry.SET_MERGING_IN_ZK);
-    if (server != null && server.getZooKeeper() != null) {
+    if (useZKAndZKIsSet(server)) {
       // After creating the merge node, wait for master to transition it
       // from PENDING_MERGE to MERGING so that we can move on. We want master
       // knows about it and won't transition any region which is merging.
@@ -399,7 +420,7 @@ public class RegionMergeTransaction {
     // clean this up.
     mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
 
-    if (server != null && server.getZooKeeper() != null) {
+    if (server != null && useZKAndZKIsSet(server)) {
       try {
         // Do one more check on the merging znode (before it is too late) in case
         // any merging region is moved somehow. If so, the znode transition will fail.
@@ -548,7 +569,13 @@ public class RegionMergeTransaction {
 
     if (services != null) {
       try {
-        services.postOpenDeployTasks(merged, server.getCatalogTracker());
+        if (useZKForAssignment) {
+          services.postOpenDeployTasks(merged, server.getCatalogTracker());
+        } else if (!services.reportRegionTransition(TransitionCode.MERGED,
+            mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
+          throw new IOException("Failed to report merged region to master: "
+            + mergedRegionInfo.getShortNameToLog());
+        }
         services.addToOnlineRegions(merged);
       } catch (KeeperException ke) {
         throw new IOException(ke);
@@ -567,43 +594,41 @@ public class RegionMergeTransaction {
    */
   void transitionZKNode(final Server server, final RegionServerServices services,
       HRegion mergedRegion) throws IOException {
-    if (server == null || server.getZooKeeper() == null) {
-      return;
-    }
-
-    // Tell master about merge by updating zk. If we fail, abort.
-    try {
-      this.znodeVersion = transitionMergingNode(server.getZooKeeper(),
-        this.mergedRegionInfo, region_a.getRegionInfo(),
-        region_b.getRegionInfo(), server.getServerName(), this.znodeVersion,
-        RS_ZK_REGION_MERGING, RS_ZK_REGION_MERGED);
-
-      long startTime = EnvironmentEdgeManager.currentTimeMillis();
-      int spins = 0;
-      // Now wait for the master to process the merge. We know it's done
-      // when the znode is deleted. The reason we keep tickling the znode is
-      // that it's possible for the master to miss an event.
-      do {
-        if (spins % 10 == 0) {
-          LOG.debug("Still waiting on the master to process the merge for "
-              + this.mergedRegionInfo.getEncodedName() + ", waited "
-              + (EnvironmentEdgeManager.currentTimeMillis() - startTime) + "ms");
-        }
-        Thread.sleep(100);
-        // When this returns -1 it means the znode doesn't exist
+    if (useZKAndZKIsSet(server)) {
+      // Tell master about merge by updating zk. If we fail, abort.
+      try {
         this.znodeVersion = transitionMergingNode(server.getZooKeeper(),
           this.mergedRegionInfo, region_a.getRegionInfo(),
           region_b.getRegionInfo(), server.getServerName(), this.znodeVersion,
-          RS_ZK_REGION_MERGED, RS_ZK_REGION_MERGED);
-        spins++;
-      } while (this.znodeVersion != -1 && !server.isStopped()
-          && !services.isStopping());
-    } catch (Exception e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
+          RS_ZK_REGION_MERGING, RS_ZK_REGION_MERGED);
+  
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+        int spins = 0;
+        // Now wait for the master to process the merge. We know it's done
+        // when the znode is deleted. The reason we keep tickling the znode is
+        // that it's possible for the master to miss an event.
+        do {
+          if (spins % 10 == 0) {
+            LOG.debug("Still waiting on the master to process the merge for "
+                + this.mergedRegionInfo.getEncodedName() + ", waited "
+                + (EnvironmentEdgeManager.currentTimeMillis() - startTime) + "ms");
+          }
+          Thread.sleep(100);
+          // When this returns -1 it means the znode doesn't exist
+          this.znodeVersion = transitionMergingNode(server.getZooKeeper(),
+            this.mergedRegionInfo, region_a.getRegionInfo(),
+            region_b.getRegionInfo(), server.getServerName(), this.znodeVersion,
+            RS_ZK_REGION_MERGED, RS_ZK_REGION_MERGED);
+          spins++;
+        } while (this.znodeVersion != -1 && !server.isStopped()
+            && !services.isStopping());
+      } catch (Exception e) {
+        if (e instanceof InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        throw new IOException("Failed telling master about merge "
+            + mergedRegionInfo.getEncodedName(), e);
       }
-      throw new IOException("Failed telling master about merge "
-          + mergedRegionInfo.getEncodedName(), e);
     }
 
     if (rsCoprocessorHost != null) {
@@ -745,8 +770,12 @@ public class RegionMergeTransaction {
       switch (je) {
 
         case SET_MERGING_IN_ZK:
-          if (server != null && server.getZooKeeper() != null) {
+          if (useZKAndZKIsSet(server)) {
             cleanZK(server, this.mergedRegionInfo);
+          } else if (services != null && !useZKForAssignment
+              && !services.reportRegionTransition(TransitionCode.MERGE_REVERTED,
+                  mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
+            return false;
           }
           break;
 
@@ -822,6 +851,10 @@ public class RegionMergeTransaction {
     return this.mergesdir;
   }
 
+  private boolean useZKAndZKIsSet(final Server server) {
+    return server != null && useZKForAssignment && server.getZooKeeper() != null;
+  }
+
   private static void cleanZK(final Server server, final HRegionInfo hri) {
     try {
       // Only delete if its in expected state; could have been hijacked.

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 2be10cd..de109f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.master.TableLockManager;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.zookeeper.KeeperException;
 
@@ -81,6 +82,16 @@ public interface RegionServerServices
   throws KeeperException, IOException;
 
   /**
+   * Notify master that a handler requests to change a region state
+   */
+  boolean reportRegionTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris);
+
+  /**
+   * Notify master that a handler requests to change a region state
+   */
+  boolean reportRegionTransition(TransitionCode code, HRegionInfo... hris);
+
+  /**
    * Returns a reference to the region server's RPC server
    */
   RpcServerInterface getRpcServer();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
index 3394ccd..50e716c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
@@ -49,8 +49,10 @@ import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -98,6 +100,7 @@ public class SplitTransaction {
   private HRegionInfo hri_b;
   private long fileSplitTimeout = 30000;
   private int znodeVersion = -1;
+  boolean useZKForAssignment;
 
   /*
    * Row to split around
@@ -281,15 +284,22 @@ public class SplitTransaction {
     // will determine whether the region is split or not in case of failures.
     // If it is successful, master will roll-forward, if not, master will rollback
     // and assign the parent region.
-    if (!testing) {
+    if (!testing && useZKForAssignment) {
       if (metaEntries == null || metaEntries.isEmpty()) {
-        MetaEditor.splitRegion(server.getCatalogTracker(),
-            parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(),
-            daughterRegions.getSecond().getRegionInfo(), server.getServerName());
+        MetaEditor.splitRegion(server.getCatalogTracker(), parent.getRegionInfo(), daughterRegions
+            .getFirst().getRegionInfo(), daughterRegions.getSecond().getRegionInfo(), server
+            .getServerName());
       } else {
-        offlineParentInMetaAndputMetaEntries(server.getCatalogTracker(),
-          parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
-              .getSecond().getRegionInfo(), server.getServerName(), metaEntries);
+        offlineParentInMetaAndputMetaEntries(server.getCatalogTracker(), parent.getRegionInfo(),
+          daughterRegions.getFirst().getRegionInfo(), daughterRegions.getSecond().getRegionInfo(),
+          server.getServerName(), metaEntries);
+      }
+    } else if (services != null && !useZKForAssignment) {
+      if (!services.reportRegionTransition(TransitionCode.SPLIT_PONR, parent.getRegionInfo(),
+        hri_a, hri_b)) {
+        // Passed PONR, let SSH clean it up
+        throw new IOException("Failed to notify master that split passed PONR: "
+            + parent.getRegionInfo().getRegionNameAsString());
       }
     }
     return daughterRegions;
@@ -299,7 +309,7 @@ public class SplitTransaction {
       final RegionServerServices services, boolean testing) throws IOException {
     // Set ephemeral SPLITTING znode up in zk.  Mocked servers sometimes don't
     // have zookeeper so don't do zk stuff if server or zookeeper is null
-    if (server != null && server.getZooKeeper() != null) {
+    if (server != null && server.getZooKeeper() != null && useZKForAssignment) {
       try {
         createNodeSplitting(server.getZooKeeper(),
           parent.getRegionInfo(), server.getServerName(), hri_a, hri_b);
@@ -307,9 +317,15 @@ public class SplitTransaction {
         throw new IOException("Failed creating PENDING_SPLIT znode on " +
           this.parent.getRegionNameAsString(), e);
       }
+    } else if (services != null && !useZKForAssignment) {
+      if (!services.reportRegionTransition(TransitionCode.READY_TO_SPLIT,
+        parent.getRegionInfo(), hri_a, hri_b)) {
+        throw new IOException("Failed to get ok from master to split "
+            + parent.getRegionNameAsString());
+      }
     }
     this.journal.add(JournalEntry.SET_SPLITTING_IN_ZK);
-    if (server != null && server.getZooKeeper() != null) {
+    if (server != null && server.getZooKeeper() != null && useZKForAssignment) {
       // After creating the split node, wait for master to transition it
       // from PENDING_SPLIT to SPLITTING so that we can move on. We want master
       // knows about it and won't transition any region which is splitting.
@@ -411,11 +427,19 @@ public class SplitTransaction {
       }
       if (services != null) {
         try {
-          // add 2nd daughter first (see HBASE-4335)
-          services.postOpenDeployTasks(b, server.getCatalogTracker());
+          if (useZKForAssignment) {
+            // add 2nd daughter first (see HBASE-4335)
+            services.postOpenDeployTasks(b, server.getCatalogTracker());
+          } else if (!services.reportRegionTransition(TransitionCode.SPLIT,
+              parent.getRegionInfo(), hri_a, hri_b)) {
+            throw new IOException("Failed to report split region to master: "
+              + parent.getRegionInfo().getShortNameToLog());
+          }
           // Should add it to OnlineRegions
           services.addToOnlineRegions(b);
-          services.postOpenDeployTasks(a, server.getCatalogTracker());
+          if (useZKForAssignment) {
+            services.postOpenDeployTasks(a, server.getCatalogTracker());
+          }
           services.addToOnlineRegions(a);
         } catch (KeeperException ke) {
           throw new IOException(ke);
@@ -471,10 +495,7 @@ public class SplitTransaction {
       }
     }
 
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().postSplit(a,b);
-    }
+    
 
     // Leaving here, the splitdir with its dross will be in place but since the
     // split was successful, just leave it; it'll be cleaned when parent is
@@ -565,6 +586,8 @@ public class SplitTransaction {
   public PairOfSameType<HRegion> execute(final Server server,
       final RegionServerServices services)
   throws IOException {
+    useZKForAssignment =
+        server == null ? true : ConfigUtil.useZKForAssignment(server.getConfiguration());
     PairOfSameType<HRegion> regions = createDaughters(server, services);
     if (this.parent.getCoprocessorHost() != null) {
       this.parent.getCoprocessorHost().preSplitAfterPONR();
@@ -576,7 +599,13 @@ public class SplitTransaction {
       final RegionServerServices services, PairOfSameType<HRegion> regions)
       throws IOException {
     openDaughters(server, services, regions.getFirst(), regions.getSecond());
-    transitionZKNode(server, services, regions.getFirst(), regions.getSecond());
+    if (server != null && server.getZooKeeper() != null && useZKForAssignment) {
+      transitionZKNode(server, services, regions.getFirst(), regions.getSecond());
+    }
+    // Coprocessor callback
+    if (this.parent.getCoprocessorHost() != null) {
+      this.parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
+    }
     return regions;
   }
 
@@ -801,8 +830,13 @@ public class SplitTransaction {
       switch(je) {
 
       case SET_SPLITTING_IN_ZK:
-        if (server != null && server.getZooKeeper() != null) {
+        if (server != null && server.getZooKeeper() != null && useZKForAssignment) {
           cleanZK(server, this.parent.getRegionInfo());
+        } else if (services != null
+            && !useZKForAssignment
+            && !services.reportRegionTransition(TransitionCode.SPLIT_REVERTED,
+              parent.getRegionInfo(), hri_a, hri_b)) {
+          return false;
         }
         break;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 54c77c7..80b8d66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -28,8 +28,10 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.zookeeper.KeeperException;
 
@@ -62,6 +64,7 @@ public class CloseRegionHandler extends EventHandler {
   // CLOSING.
   private final boolean zk;
   private ServerName destination;
+  private final boolean useZKForAssignment;
 
   // This is executed after receiving an CLOSE RPC from the master.
   public CloseRegionHandler(final Server server,
@@ -100,9 +103,8 @@ public class CloseRegionHandler extends EventHandler {
     this(server, rsServices, regionInfo, abort, zk, versionOfClosingNode, eventType, null);
   }
 
-    protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, HRegionInfo regionInfo,
-      boolean abort, final boolean zk, final int versionOfClosingNode,
+  protected CloseRegionHandler(final Server server, final RegionServerServices rsServices,
+      HRegionInfo regionInfo, boolean abort, final boolean zk, final int versionOfClosingNode,
       EventType eventType, ServerName destination) {
     super(server, eventType);
     this.server = server;
@@ -112,6 +114,7 @@ public class CloseRegionHandler extends EventHandler {
     this.zk = zk;
     this.expectedVersion = versionOfClosingNode;
     this.destination = destination;
+    useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
   }
 
   public HRegionInfo getRegionInfo() {
@@ -137,7 +140,8 @@ public class CloseRegionHandler extends EventHandler {
 
       // Close the region
       try {
-        if (zk && !ZKAssign.checkClosingState(server.getZooKeeper(), regionInfo, expectedVersion)){
+        if (zk && useZKForAssignment
+            && !ZKAssign.checkClosingState(server.getZooKeeper(), regionInfo, expectedVersion)) {
           // bad znode state
           return; // We're node deleting the znode, but it's not ours...
         }
@@ -162,16 +166,18 @@ public class CloseRegionHandler extends EventHandler {
       }
 
       this.rsServices.removeFromOnlineRegions(region, destination);
-
-      if (this.zk) {
-        if (setClosedState(this.expectedVersion, region)) {
-          LOG.debug("Set closed state in zk for " + name + " on " + this.server.getServerName());
-        } else {
-          LOG.debug("Set closed state in zk UNSUCCESSFUL for " + name + " on " +
-            this.server.getServerName());
+      if (!useZKForAssignment) {
+        rsServices.reportRegionTransition(TransitionCode.CLOSED, regionInfo);
+      } else {
+        if (this.zk) {
+          if (setClosedState(this.expectedVersion, region)) {
+            LOG.debug("Set closed state in zk for " + name + " on " + this.server.getServerName());
+          } else {
+            LOG.debug("Set closed state in zk UNSUCCESSFUL for " + name + " on "
+                + this.server.getServerName());
+          }
         }
       }
-
       // Done!  Region is closed on this RS
       LOG.debug("Closed " + region.getRegionNameAsString());
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
index fb689f9..ec78aa4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
@@ -30,10 +30,12 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
@@ -61,6 +63,8 @@ public class OpenRegionHandler extends EventHandler {
   //version of the offline node that was set by the master
   private volatile int versionOfOfflineNode = -1;
 
+  private final boolean useZKForAssignment;
+
   public OpenRegionHandler(final Server server,
       final RegionServerServices rsServices, HRegionInfo regionInfo,
       HTableDescriptor htd) {
@@ -88,6 +92,7 @@ public class OpenRegionHandler extends EventHandler {
     assignmentTimeout = this.server.getConfiguration().
       getInt(AssignmentManager.ASSIGNMENT_TIMEOUT,
         AssignmentManager.DEFAULT_ASSIGNMENT_TIMEOUT_DEFAULT);
+    useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
   }
 
   public HRegionInfo getRegionInfo() {
@@ -128,7 +133,8 @@ public class OpenRegionHandler extends EventHandler {
         return;
       }
 
-      if (!transitionZookeeperOfflineToOpening(encodedName, versionOfOfflineNode)) {
+      if (useZKForAssignment
+          && !transitionZookeeperOfflineToOpening(encodedName, versionOfOfflineNode)) {
         LOG.warn("Region was hijacked? Opening cancelled for encodedName=" + encodedName);
         // This is a desperate attempt: the znode is unlikely to be ours. But we can't do more.
         return;
@@ -142,7 +148,7 @@ public class OpenRegionHandler extends EventHandler {
       }
 
       boolean failed = true;
-      if (tickleOpening("post_region_open")) {
+      if (!useZKForAssignment || tickleOpening("post_region_open")) {
         if (updateMeta(region)) {
           failed = false;
         }
@@ -153,7 +159,7 @@ public class OpenRegionHandler extends EventHandler {
       }
 
 
-      if (!isRegionStillOpening() || !transitionToOpened(region)) {
+      if (!isRegionStillOpening() || (useZKForAssignment && !transitionToOpened(region))) {
         // If we fail to transition to opened, it's because of one of two cases:
         //    (a) we lost our ZK lease
         // OR (b) someone else opened the region before us
@@ -218,10 +224,16 @@ public class OpenRegionHandler extends EventHandler {
           cleanupFailedOpen(region);
         }
       } finally {
+        if (!useZKForAssignment) {
+          rsServices.reportRegionTransition(TransitionCode.FAILED_OPEN, regionInfo);
+        } else {
         // Even if cleanupFailed open fails we need to do this transition
         // See HBASE-7698
         tryTransitionFromOpeningToFailedOpen(regionInfo);
+        }
       }
+    } else if (!useZKForAssignment) {
+      rsServices.reportRegionTransition(TransitionCode.FAILED_OPEN, regionInfo);
     } else {
       // If still transition to OPENING is not done, we need to transition znode
       // to FAILED_OPEN
@@ -262,7 +274,9 @@ public class OpenRegionHandler extends EventHandler {
       if (elapsed > period) {
         // Only tickle OPENING if postOpenDeployTasks is taking some time.
         lastUpdate = now;
-        tickleOpening = tickleOpening("post_open_deploy");
+        if (useZKForAssignment) {
+          tickleOpening = tickleOpening("post_open_deploy");
+        }
       }
       synchronized (signaller) {
         try {
@@ -467,12 +481,20 @@ public class OpenRegionHandler extends EventHandler {
           this.server.getConfiguration(),
           this.rsServices,
         new CancelableProgressable() {
-          public boolean progress() {
-            // We may lose the znode ownership during the open.  Currently its
-            // too hard interrupting ongoing region open.  Just let it complete
-            // and check we still have the znode after region open.
-            return tickleOpening("open_region_progress");
-          }
+              public boolean progress() {
+                if (useZKForAssignment) {
+                  // We may lose the znode ownership during the open. Currently its
+                  // too hard interrupting ongoing region open. Just let it complete
+                  // and check we still have the znode after region open.
+                  // if tickle failed, we need to cancel opening region.
+                  return tickleOpening("open_region_progress");
+                }
+                if (!isRegionStillOpening()) {
+                  LOG.warn("Open region aborted since it isn't opening any more");
+                  return false;
+                }
+                return true;
+              }
         });
     } catch (Throwable t) {
       // We failed open. Our caller will see the 'null' return value

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigUtil.java
new file mode 100644
index 0000000..2183ee9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigUtil.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Some configuration related utilities
+ */
+@InterfaceAudience.Private
+public class ConfigUtil {
+
+  public static boolean useZKForAssignment(Configuration conf) {
+    // To change the default, please also update ZooKeeperWatcher.java
+    return conf.getBoolean("hbase.assignment.usezk", true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 50ad030..e399ad5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -153,7 +154,7 @@ class MockRegionServerServices implements RegionServerServices {
 
   @Override
   public Configuration getConfiguration() {
-    return null;
+    return zkw == null ? null : zkw.getConfiguration();
   }
 
   @Override
@@ -227,4 +228,16 @@ class MockRegionServerServices implements RegionServerServices {
     // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public boolean reportRegionTransition(TransitionCode code, long openSeqNum,
+      HRegionInfo... hris) {
+    return false;
+  }
+
+  @Override
+  public boolean reportRegionTransition(TransitionCode code,
+      HRegionInfo... hris) {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
index 9a51729..1a74c31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
@@ -79,6 +79,7 @@ public class TestDrainingServer {
   
   @BeforeClass
   public static void beforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     TEST_UTIL.startMiniZKCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 46b02a0..46a4d79 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -491,7 +492,9 @@ public class TestScannersFromClientSide {
       RegionStates states = master.getAssignmentManager().getRegionStates();
       states.regionOffline(hri);
       states.updateRegionState(hri, State.OPENING);
-      ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
+      if (ConfigUtil.useZKForAssignment(TEST_UTIL.getConfiguration())) {
+        ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
+      }
       ProtobufUtil.openRegion(rs, rs.getServerName(), hri);
       startTime = EnvironmentEdgeManager.currentTimeMillis();
       while (true) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 45b9885..db7a75b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -560,4 +561,15 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public ServerNonceManager getNonceManager() {
     return null;
   }
+
+  @Override
+  public boolean reportRegionTransition(TransitionCode code, HRegionInfo... hris) {
+    return false;
+  }
+
+  @Override
+  public boolean reportRegionTransition(TransitionCode code, long openSeqNum,
+      HRegionInfo... hris) {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
index 1ba8eac..09ac83d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
@@ -120,6 +120,7 @@ public class TestAssignmentManager {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
+    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     HTU.startMiniZKCluster();
   }
 
@@ -139,6 +140,7 @@ public class TestAssignmentManager {
     this.server = Mockito.mock(Server.class);
     Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
     Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
+    Mockito.when(server.getCatalogTracker()).thenReturn(null);
     this.watcher =
       new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
     Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index 8ffc9fa..eb15240 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@@ -75,11 +76,10 @@ import org.junit.experimental.categories.Category;
 public class TestAssignmentManagerOnCluster {
   private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final static Configuration conf = TEST_UTIL.getConfiguration();
+  final static Configuration conf = TEST_UTIL.getConfiguration();
   private static HBaseAdmin admin;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  static void setupOnce() throws Exception {
     // Using the our load balancer to control region plans
     conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
       MyLoadBalancer.class, LoadBalancer.class);
@@ -92,6 +92,13 @@ public class TestAssignmentManagerOnCluster {
     admin = TEST_UTIL.getHBaseAdmin();
   }
 
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Use ZK for region assignment
+    conf.setBoolean("hbase.assignment.usezk", true);
+    setupOnce();
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
@@ -554,16 +561,18 @@ public class TestAssignmentManagerOnCluster {
       }
       am.regionOffline(hri);
       ZooKeeperWatcher zkw = TEST_UTIL.getHBaseCluster().getMaster().getZooKeeper();
-      am.getRegionStates().updateRegionState(hri, State.OFFLINE);
-      ZKAssign.createNodeOffline(zkw, hri, destServerName);
-      ZKAssign.transitionNodeOpening(zkw, hri, destServerName);
-
-      // Wait till the event is processed and the region is in transition
-      long timeoutTime = System.currentTimeMillis() + 20000;
-      while (!am.getRegionStates().isRegionInTransition(hri)) {
-        assertTrue("Failed to process ZK opening event in time",
-          System.currentTimeMillis() < timeoutTime);
-        Thread.sleep(100);
+      am.getRegionStates().updateRegionState(hri, State.PENDING_OPEN, destServerName);
+      if (ConfigUtil.useZKForAssignment(conf)) {
+        ZKAssign.createNodeOffline(zkw, hri, destServerName);
+        ZKAssign.transitionNodeOpening(zkw, hri, destServerName);
+  
+        // Wait till the event is processed and the region is in transition
+        long timeoutTime = System.currentTimeMillis() + 20000;
+        while (!am.getRegionStates().isRegionInTransition(hri)) {
+          assertTrue("Failed to process ZK opening event in time",
+            System.currentTimeMillis() < timeoutTime);
+          Thread.sleep(100);
+        }
       }
 
       am.getZKTable().setDisablingTable(table);
@@ -697,8 +706,6 @@ public class TestAssignmentManagerOnCluster {
       ServerName serverName = master.getAssignmentManager().
         getRegionStates().getRegionServerOfRegion(hri);
       TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
-      assertFalse("Region should be assigned on a new region server",
-        oldServerName.equals(serverName));
     } finally {
       MyRegionObserver.postOpenEnabled.set(false);
       TEST_UTIL.deleteTable(Bytes.toBytes(table));

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index 07be41d..7a786f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
@@ -47,6 +48,8 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -159,6 +162,7 @@ public class TestMasterFailover {
 
     // Create config to use for this cluster
     Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.assignment.usezk", true);
 
     // Start the cluster
     HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
@@ -494,6 +498,7 @@ public class TestMasterFailover {
     // Create and start the cluster
     HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setBoolean("hbase.assignment.usezk", true);
 
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 2);
@@ -932,6 +937,7 @@ public class TestMasterFailover {
     HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setInt("hbase.master.info.port", -1);
+    conf.setBoolean("hbase.assignment.usezk", true);
 
     TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@@ -984,6 +990,7 @@ public class TestMasterFailover {
 
     // Create config to use for this cluster
     Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.assignment.usezk", true);
 
     // Start the cluster
     final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
@@ -1152,5 +1159,101 @@ public class TestMasterFailover {
     // Stop the cluster
     TEST_UTIL.shutdownMiniCluster();
   }
+
+  /**
+   * Test region in pending_open/close when master failover
+   */
+  @Test (timeout=180000)
+  public void testPendingOpenOrCloseWhenMasterFailover() throws Exception {
+    final int NUM_MASTERS = 1;
+    final int NUM_RS = 1;
+
+    // Create config to use for this cluster
+    Configuration conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.assignment.usezk", false);
+
+    // Start the cluster
+    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
+    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
+    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+    log("Cluster started");
+
+    // get all the master threads
+    List<MasterThread> masterThreads = cluster.getMasterThreads();
+    assertEquals(1, masterThreads.size());
+
+    // only one master thread, let's wait for it to be initialized
+    assertTrue(cluster.waitForActiveAndReadyMaster());
+    HMaster master = masterThreads.get(0).getMaster();
+    assertTrue(master.isActiveMaster());
+    assertTrue(master.isInitialized());
+
+    // Create a table with a region online
+    HTable onlineTable = TEST_UTIL.createTable("onlineTable", "family");
+
+    // Create a table in META, so it has a region offline
+    HTableDescriptor offlineTable = new HTableDescriptor(
+      TableName.valueOf(Bytes.toBytes("offlineTable")));
+    offlineTable.addFamily(new HColumnDescriptor(Bytes.toBytes("family")));
+
+    FileSystem filesystem = FileSystem.get(conf);
+    Path rootdir = FSUtils.getRootDir(conf);
+    FSTableDescriptors fstd = new FSTableDescriptors(filesystem, rootdir);
+    fstd.createTableDescriptor(offlineTable);
+
+    HRegionInfo hriOffline = new HRegionInfo(offlineTable.getTableName(), null, null);
+    createRegion(hriOffline, rootdir, conf, offlineTable);
+    MetaEditor.addRegionToMeta(master.getCatalogTracker(), hriOffline);
+
+    log("Regions in hbase:meta and namespace have been created");
+
+    // at this point we only expect 3 regions to be assigned out
+    // (catalogs and namespace, + 1 online region)
+    assertEquals(3, cluster.countServedRegions());
+    HRegionInfo hriOnline = onlineTable.getRegionLocation("").getRegionInfo();
+
+    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+    RegionStateStore stateStore = master.getAssignmentManager().getRegionStateStore();
+
+    // Put the online region in pending_close. It is actually already opened.
+    // This is to simulate that the region close RPC is not sent out before failover
+    RegionState oldState = regionStates.getRegionState(hriOnline);
+    RegionState newState = new RegionState(hriOnline, State.PENDING_CLOSE, oldState.getServerName());
+    stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
+
+    // Put the offline region in pending_open. It is actually not opened yet.
+    // This is to simulate that the region open RPC is not sent out before failover
+    oldState = new RegionState(hriOffline, State.OFFLINE);
+    newState = new RegionState(hriOffline, State.PENDING_OPEN, newState.getServerName());
+    stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
+
+    // Stop the master
+    log("Aborting master");
+    cluster.abortMaster(0);
+    cluster.waitOnMaster(0);
+    log("Master has aborted");
+
+    // Start up a new master
+    log("Starting up a new master");
+    master = cluster.startMaster().getMaster();
+    log("Waiting for master to be ready");
+    cluster.waitForActiveAndReadyMaster();
+    log("Master is ready");
+
+    // Wait till no region in transition any more
+    master.getAssignmentManager().waitUntilNoRegionsInTransition(60000);
+
+    // Get new region states since master restarted
+    regionStates = master.getAssignmentManager().getRegionStates();
+
+    // Both pending_open (RPC sent/not yet) regions should be online
+    assertTrue(regionStates.isRegionOnline(hriOffline));
+    assertTrue(regionStates.isRegionOnline(hriOnline));
+
+    log("Done with verification, shutting down cluster");
+
+    // Done, shutdown the cluster
+    TEST_UTIL.shutdownMiniCluster();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
index f68c015..da1df20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
@@ -65,6 +65,7 @@ public class TestOpenedRegionHandler {
   @Before
   public void setUp() throws Exception {
     conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.assignment.usezk", true);
     TEST_UTIL = HBaseTestingUtility.createLocalHTU(conf);
   }
 
@@ -80,6 +81,7 @@ public class TestOpenedRegionHandler {
     // Start the cluster
     log("Starting cluster");
     conf = HBaseConfiguration.create();
+    conf.setBoolean("hbase.assignment.usezk", true);
     resetConf = conf;
     conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
     conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 5000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
index 2cda4e9..372c495 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
@@ -58,6 +58,7 @@ public class TestRestartCluster {
 
   @Test (timeout=300000) public void testRestartClusterAfterKill()
   throws Exception {
+    UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     UTIL.startMiniZKCluster();
     ZooKeeperWatcher zooKeeper =
       new ZooKeeperWatcher(UTIL.getConfiguration(), "cluster1", null, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
index a923d49..d3bad28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
@@ -71,6 +71,7 @@ public class TestZKBasedOpenCloseRegion {
 
   @BeforeClass public static void beforeAllTests() throws Exception {
     Configuration c = TEST_UTIL.getConfiguration();
+    c.setBoolean("hbase.assignment.usezk", true);
     c.setBoolean("dfs.support.append", true);
     c.setInt("hbase.regionserver.info.port", 0);
     TEST_UTIL.startMiniCluster(2);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java
new file mode 100644
index 0000000..83d33c5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This tests AssignmentManager with a testing cluster.
+ */
+@Category(MediumTests.class)
+public class TestZKLessAMOnCluster extends TestAssignmentManagerOnCluster {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Don't use ZK for region assignment
+    conf.setBoolean("hbase.assignment.usezk", false);
+    setupOnce();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TestAssignmentManagerOnCluster.tearDownAfterClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index 25b6a59..f2788fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConfigUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
@@ -105,6 +106,7 @@ public class TestEndToEndSplitTransaction {
         .getRegionName();
     HRegion region = server.getRegion(regionName);
     SplitTransaction split = new SplitTransaction(region, splitRow);
+    split.useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
     split.prepare();
 
     // 1. phase I
@@ -138,8 +140,9 @@ public class TestEndToEndSplitTransaction {
     assertTrue(test(con, tableName, lastRow, server));
 
     // 4. phase III
-    split.transitionZKNode(server, server, regions.getFirst(),
-        regions.getSecond());
+    if (split.useZKForAssignment) {
+      split.transitionZKNode(server, server, regions.getFirst(), regions.getSecond());
+    }
     assertTrue(test(con, tableName, firstRow, server));
     assertTrue(test(con, tableName, lastRow, server));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index aac801e..cc0a123 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -31,7 +31,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -39,6 +38,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -47,14 +48,13 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.junit.AfterClass;
@@ -86,13 +86,12 @@ public class TestRegionMergeTransactionOnCluster {
 
   private static int waitTime = 60 * 1000;
 
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private static HMaster master;
   private static HBaseAdmin admin;
 
-  @BeforeClass
-  public static void beforeAllTests() throws Exception {
+  static void setupOnce() throws Exception {
     // Start a cluster
     TEST_UTIL.startMiniCluster(NB_SERVERS);
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@@ -101,6 +100,13 @@ public class TestRegionMergeTransactionOnCluster {
     admin = TEST_UTIL.getHBaseAdmin();
   }
 
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    // Use ZK for region assignment
+    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
+    setupOnce();
+  }
+
   @AfterClass
   public static void afterAllTests() throws Exception {
     TEST_UTIL.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 8ffc719..799a277 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -65,6 +65,7 @@ public class TestRegionServerNoMaster {
 
   @BeforeClass
   public static void before() throws Exception {
+    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
     HTU.startMiniCluster(NB_SERVERS);
     final byte[] tableName = Bytes.toBytes(TestRegionServerNoMaster.class.getSimpleName());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index af15d78..b547011 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -114,19 +114,24 @@ public class TestSplitTransactionOnCluster {
   private static volatile boolean secondSplit = false;
   private static volatile boolean callRollBack = false;
   private static volatile boolean firstSplitCompleted = false;
+  private static boolean useZKForAssignment = true;
 
-  private static final HBaseTestingUtility TESTING_UTIL =
+  static final HBaseTestingUtility TESTING_UTIL =
     new HBaseTestingUtility();
 
-  @BeforeClass public static void before() throws Exception {
+  static void setupOnce() throws Exception {
     TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
-    // Needed because some tests have splits happening on RS that are killed
-    // We don't want to wait 3min for the master to figure it out
-    TESTING_UTIL.getConfiguration().setInt(
-        "hbase.master.assignment.timeoutmonitor.timeout", 4000);
+    useZKForAssignment =
+        TESTING_UTIL.getConfiguration().getBoolean("hbase.assignment.usezk", false);
     TESTING_UTIL.startMiniCluster(NB_SERVERS);
   }
 
+  @BeforeClass public static void before() throws Exception {
+    // Use ZK for region assignment
+    TESTING_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
+    setupOnce();
+  }
+
   @AfterClass public static void after() throws Exception {
     TESTING_UTIL.shutdownMiniCluster();
   }
@@ -173,6 +178,12 @@ public class TestSplitTransactionOnCluster {
   public void testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack() throws Exception {
     final TableName tableName =
         TableName.valueOf("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack");
+
+    if (!useZKForAssignment) {
+      // This test doesn't apply if not using ZK for assignment
+      return;
+    }
+
     try {
       // Create table then get the single region for our new table.
       HTable t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf"));
@@ -349,42 +360,46 @@ public class TestSplitTransactionOnCluster {
       AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
       // Now try splitting and it should work.
       split(hri, server, regionCount);
-      // Get daughters
-      List<HRegion> daughters = checkAndGetDaughters(tableName);
-      // Assert the ephemeral node is up in zk.
+        // Assert the ephemeral node is up in zk.
       String path = ZKAssign.getNodeName(TESTING_UTIL.getZooKeeperWatcher(),
         hri.getEncodedName());
       RegionTransition rt = null;
       Stat stats = null;
-      // Wait till the znode moved to SPLIT
-      for (int i=0; i<100; i++) {
-        stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
-        rt = RegionTransition.parseFrom(ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
-          hri.getEncodedName()));
-        if (rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)) break;
-        Thread.sleep(100);
+      List<HRegion> daughters = null;
+      if (useZKForAssignment) {
+        daughters = checkAndGetDaughters(tableName);
+
+        // Wait till the znode moved to SPLIT
+        for (int i=0; i<100; i++) {
+          stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
+          rt = RegionTransition.parseFrom(ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
+            hri.getEncodedName()));
+          if (rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)) break;
+          Thread.sleep(100);
+        }
+        LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
+        assertTrue(rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT));
+        // Now crash the server
+        cluster.abortRegionServer(tableRegionIndex);
       }
-      LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
-      assertTrue(rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT));
-      // Now crash the server
-      cluster.abortRegionServer(tableRegionIndex);
       waitUntilRegionServerDead();
-      awaitDaughters(tableName, daughters.size());
+      awaitDaughters(tableName, 2);
+      if (useZKForAssignment) {
+        regions = cluster.getRegions(tableName);
+        for (HRegion r: regions) {
+          assertTrue(daughters.contains(r));
+        }
 
-      // Assert daughters are online.
-      regions = cluster.getRegions(tableName);
-      for (HRegion r: regions) {
-        assertTrue(daughters.contains(r));
-      }
-      // Finally assert that the ephemeral SPLIT znode was cleaned up.
-      for (int i=0; i<100; i++) {
-        // wait a bit (10s max) for the node to disappear
-        stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
-        if (stats == null) break;
-        Thread.sleep(100);
+        // Finally assert that the ephemeral SPLIT znode was cleaned up.
+        for (int i=0; i<100; i++) {
+          // wait a bit (10s max) for the node to disappear
+          stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
+          if (stats == null) break;
+          Thread.sleep(100);
+        }
+        LOG.info("EPHEMERAL NODE AFTER SERVER ABORT, path=" + path + ", stats=" + stats);
+        assertTrue(stats == null);
       }
-      LOG.info("EPHEMERAL NODE AFTER SERVER ABORT, path=" + path + ", stats=" + stats);
-      assertTrue(stats == null);
     } finally {
       // Set this flag back.
       AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
@@ -407,6 +422,8 @@ public class TestSplitTransactionOnCluster {
 
     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
 
+    RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
+
     // Turn off balancer so it doesn't cut in and mess up our placements.
     this.admin.setBalancerRunning(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
@@ -421,8 +438,12 @@ public class TestSplitTransactionOnCluster {
       // Insert into zk a blocking znode, a znode of same name as region
       // so it gets in way of our splitting.
       ServerName fakedServer = ServerName.valueOf("any.old.server", 1234, -1);
-      ZKAssign.createNodeClosing(TESTING_UTIL.getZooKeeperWatcher(),
-        hri, fakedServer);
+      if (useZKForAssignment) {
+        ZKAssign.createNodeClosing(TESTING_UTIL.getZooKeeperWatcher(),
+          hri, fakedServer);
+      } else {
+        regionStates.updateRegionState(hri, RegionState.State.CLOSING);
+      }
       // Now try splitting.... should fail.  And each should successfully
       // rollback.
       this.admin.split(hri.getRegionNameAsString());
@@ -433,9 +454,13 @@ public class TestSplitTransactionOnCluster {
         Thread.sleep(100);
         assertEquals(regionCount, ProtobufUtil.getOnlineRegions(server).size());
       }
-      // Now clear the zknode
-      ZKAssign.deleteClosingNode(TESTING_UTIL.getZooKeeperWatcher(),
-        hri, fakedServer);
+      if (useZKForAssignment) {
+        // Now clear the zknode
+        ZKAssign.deleteClosingNode(TESTING_UTIL.getZooKeeperWatcher(),
+          hri, fakedServer);
+      } else {
+        regionStates.regionOnline(hri, server.getServerName());
+      }
       // Now try splitting and it should work.
       split(hri, server, regionCount);
       // Get daughters
@@ -627,6 +652,11 @@ public class TestSplitTransactionOnCluster {
       KeeperException, DeserializationException, ServiceException {
     final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
 
+    if (!useZKForAssignment) {
+      // This test doesn't apply if not using ZK for assignment
+      return;
+    }
+
     // Create table then get the single region for our new table.
     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
     List<HRegion> regions = cluster.getRegions(tableName);
@@ -786,7 +816,7 @@ public class TestSplitTransactionOnCluster {
    * @throws InterruptedException
    * @throws KeeperException
    */
-  @Test
+  @Test(timeout = 60000)
   public void testSplitBeforeSettingSplittingInZK() throws Exception,
       InterruptedException, KeeperException {
     testSplitBeforeSettingSplittingInZKInternals();
@@ -857,7 +887,7 @@ public class TestSplitTransactionOnCluster {
    * If a table has regions that have no store files in a region, they should split successfully
    * into two regions with no store files.
    */
-  @Test
+  @Test(timeout = 60000)
   public void testSplitRegionWithNoStoreFiles()
       throws Exception {
     final TableName tableName =
@@ -1120,13 +1150,21 @@ public class TestSplitTransactionOnCluster {
   private void split(final HRegionInfo hri, final HRegionServer server, final int regionCount)
       throws IOException, InterruptedException {
     this.admin.split(hri.getRegionNameAsString());
-    for (int i = 0; ProtobufUtil.getOnlineRegions(server).size() <= regionCount && i < 300; i++) {
-      LOG.debug("Waiting on region to split");
-      Thread.sleep(100);
-    }
+    try {
+      for (int i = 0; ProtobufUtil.getOnlineRegions(server).size() <= regionCount && i < 300; i++) {
+        LOG.debug("Waiting on region to split");
+        Thread.sleep(100);
+      }
 
-    assertFalse("Waited too long for split",
+      assertFalse("Waited too long for split",
         ProtobufUtil.getOnlineRegions(server).size() <= regionCount);
+    } catch (RegionServerStoppedException e) {
+      if (useZKForAssignment) {
+        // If not using ZK for assignment, the exception may be expected.
+        LOG.error(e);
+        throw e;
+      }
+    }
   }
 
   /**


[07/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [3/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [3/8]

HBASE-11725 Backport failover checking change to 1.0 (Jimmy Xiang)


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

Branch: refs/heads/0.98
Commit: 7852df6069a8031f5c68fa48278bc16f60290a74
Parents: cbc7860
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:03:35 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/AssignmentManager.java | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7852df60/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index b6fd2cf..60e0afa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -538,8 +538,10 @@ public class AssignmentManager extends ZooKeeperListener {
       }
     } else {
       // If any one region except meta is assigned, it's a failover.
-      for (HRegionInfo hri : regionStates.getRegionAssignments().keySet()) {
-        if (!hri.isMetaTable()) {
+      Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
+      for (Map.Entry<HRegionInfo, ServerName> en : regionStates.getRegionAssignments().entrySet()) {
+        HRegionInfo hri = en.getKey();
+        if (!hri.isMetaTable() && onlineServers.contains(en.getValue())) {
           LOG.debug("Found " + hri + " out on cluster");
           failover = true;
           break;


[09/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [5/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [5/8]

HBASE-11531 RegionStates for regions under region-in-transition znode are not updated on startup (Jimmy Xiang)


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

Branch: refs/heads/0.98
Commit: f51b68b280561fbd52ecd5afd514877954d1e190
Parents: 44f646c
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:03:56 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/AssignmentManager.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f51b68b2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 1700a56..b657ed4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -499,7 +499,7 @@ public class AssignmentManager extends ZooKeeperListener {
 
     if (!useZKForAssignment) {
       // Not use ZK for assignment any more, remove the ZNode
-      ZKUtil.deleteNodeFailSilent(watcher, watcher.assignmentZNode);
+      ZKUtil.deleteNodeRecursively(watcher, watcher.assignmentZNode);
     }
     recoverTableInDisablingState();
     recoverTableInEnablingState();
@@ -743,7 +743,8 @@ public class AssignmentManager extends ZooKeeperListener {
     final String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
     LOG.info("Processing " + prettyPrintedRegionName + " in state: " + et);
 
-    if (regionStates.isRegionInTransition(encodedName)) {
+    if (regionStates.isRegionInTransition(encodedName)
+        && (regionInfo.isMetaRegion() || !useZKForAssignment)) {
       LOG.info("Processed region " + prettyPrintedRegionName + " in state: "
         + et + ", does nothing since the region is already in transition "
         + regionStates.getRegionTransitionState(encodedName));


[10/12] git commit: HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [6/8]

Posted by ap...@apache.org.
HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [6/8]

HBASE-11814 TestAssignmentManager.testCloseFailed and testOpenCloseRacing is flaky (Virag Kothari)


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

Branch: refs/heads/0.98
Commit: e09adb8a51df7fcb505f0cb19621d2c90716f391
Parents: f51b68b
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Aug 26 08:01:10 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 26 08:04:07 2014 -0700

----------------------------------------------------------------------
 .../hbase/master/TestAssignmentManagerOnCluster.java      | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e09adb8a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index 4e96e98..c3a507d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -380,7 +380,7 @@ public class TestAssignmentManagerOnCluster {
       master.assignRegion(hri);
       AssignmentManager am = master.getAssignmentManager();
       assertTrue(am.waitForAssignment(hri));
-
+      
       MyRegionObserver.preCloseEnabled.set(true);
       am.unassign(hri);
       RegionState state = am.getRegionStates().getRegionState(hri);
@@ -400,7 +400,7 @@ public class TestAssignmentManagerOnCluster {
 
       ServerName serverName = master.getAssignmentManager().
         getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
+      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
     } finally {
       MyRegionObserver.preCloseEnabled.set(false);
       TEST_UTIL.deleteTable(Bytes.toBytes(table));
@@ -427,6 +427,8 @@ public class TestAssignmentManagerOnCluster {
       master.assignRegion(hri);
       AssignmentManager am = master.getAssignmentManager();
       assertTrue(am.waitForAssignment(hri));
+      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
 
       MyRegionObserver.preCloseEnabled.set(true);
       am.unassign(hri);
@@ -611,6 +613,8 @@ public class TestAssignmentManagerOnCluster {
       master.assignRegion(hri);
       AssignmentManager am = master.getAssignmentManager();
       assertTrue(am.waitForAssignment(hri));
+      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
 
       MyRegionObserver.postCloseEnabled.set(true);
       am.unassign(hri);
@@ -707,7 +711,7 @@ public class TestAssignmentManagerOnCluster {
 
       ServerName serverName = master.getAssignmentManager().
         getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
+      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
     } finally {
       MyRegionObserver.postOpenEnabled.set(false);
       TEST_UTIL.deleteTable(Bytes.toBytes(table));


[04/12] HBASE-11546 Backport ZK-less region assignment to 0.98 (Virag Kothari) [1/8]

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e6ffa86e/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java
index 261a6f2..6e0bf59 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RegionServerStatusProtos.java
@@ -4962,6 +4962,2808 @@ public final class RegionServerStatusProtos {
     // @@protoc_insertion_point(class_scope:GetLastFlushedSequenceIdResponse)
   }
 
+  public interface RegionTransitionOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .RegionTransition.TransitionCode transition_code = 1;
+    /**
+     * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+     */
+    boolean hasTransitionCode();
+    /**
+     * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode getTransitionCode();
+
+    // repeated .RegionInfo region_info = 2;
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+
+    // optional uint64 open_seq_num = 3;
+    /**
+     * <code>optional uint64 open_seq_num = 3;</code>
+     *
+     * <pre>
+     ** For newly opened region, the open seq num is needed 
+     * </pre>
+     */
+    boolean hasOpenSeqNum();
+    /**
+     * <code>optional uint64 open_seq_num = 3;</code>
+     *
+     * <pre>
+     ** For newly opened region, the open seq num is needed 
+     * </pre>
+     */
+    long getOpenSeqNum();
+  }
+  /**
+   * Protobuf type {@code RegionTransition}
+   */
+  public static final class RegionTransition extends
+      com.google.protobuf.GeneratedMessage
+      implements RegionTransitionOrBuilder {
+    // Use RegionTransition.newBuilder() to construct.
+    private RegionTransition(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private RegionTransition(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final RegionTransition defaultInstance;
+    public static RegionTransition getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public RegionTransition getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private RegionTransition(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode value = org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                transitionCode_ = value;
+              }
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000002;
+              openSeqNum_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_RegionTransition_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_RegionTransition_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.class, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<RegionTransition> PARSER =
+        new com.google.protobuf.AbstractParser<RegionTransition>() {
+      public RegionTransition parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new RegionTransition(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<RegionTransition> getParserForType() {
+      return PARSER;
+    }
+
+    /**
+     * Protobuf enum {@code RegionTransition.TransitionCode}
+     */
+    public enum TransitionCode
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>OPENED = 0;</code>
+       */
+      OPENED(0, 0),
+      /**
+       * <code>FAILED_OPEN = 1;</code>
+       */
+      FAILED_OPEN(1, 1),
+      /**
+       * <code>CLOSED = 2;</code>
+       *
+       * <pre>
+       ** No failed_close, in which case region server will abort 
+       * </pre>
+       */
+      CLOSED(2, 2),
+      /**
+       * <code>READY_TO_SPLIT = 3;</code>
+       *
+       * <pre>
+       ** Ask master for ok to split/merge region(s) 
+       * </pre>
+       */
+      READY_TO_SPLIT(3, 3),
+      /**
+       * <code>READY_TO_MERGE = 4;</code>
+       */
+      READY_TO_MERGE(4, 4),
+      /**
+       * <code>SPLIT_PONR = 5;</code>
+       */
+      SPLIT_PONR(5, 5),
+      /**
+       * <code>MERGE_PONR = 6;</code>
+       */
+      MERGE_PONR(6, 6),
+      /**
+       * <code>SPLIT = 7;</code>
+       */
+      SPLIT(7, 7),
+      /**
+       * <code>MERGED = 8;</code>
+       */
+      MERGED(8, 8),
+      /**
+       * <code>SPLIT_REVERTED = 9;</code>
+       */
+      SPLIT_REVERTED(9, 9),
+      /**
+       * <code>MERGE_REVERTED = 10;</code>
+       */
+      MERGE_REVERTED(10, 10),
+      ;
+
+      /**
+       * <code>OPENED = 0;</code>
+       */
+      public static final int OPENED_VALUE = 0;
+      /**
+       * <code>FAILED_OPEN = 1;</code>
+       */
+      public static final int FAILED_OPEN_VALUE = 1;
+      /**
+       * <code>CLOSED = 2;</code>
+       *
+       * <pre>
+       ** No failed_close, in which case region server will abort 
+       * </pre>
+       */
+      public static final int CLOSED_VALUE = 2;
+      /**
+       * <code>READY_TO_SPLIT = 3;</code>
+       *
+       * <pre>
+       ** Ask master for ok to split/merge region(s) 
+       * </pre>
+       */
+      public static final int READY_TO_SPLIT_VALUE = 3;
+      /**
+       * <code>READY_TO_MERGE = 4;</code>
+       */
+      public static final int READY_TO_MERGE_VALUE = 4;
+      /**
+       * <code>SPLIT_PONR = 5;</code>
+       */
+      public static final int SPLIT_PONR_VALUE = 5;
+      /**
+       * <code>MERGE_PONR = 6;</code>
+       */
+      public static final int MERGE_PONR_VALUE = 6;
+      /**
+       * <code>SPLIT = 7;</code>
+       */
+      public static final int SPLIT_VALUE = 7;
+      /**
+       * <code>MERGED = 8;</code>
+       */
+      public static final int MERGED_VALUE = 8;
+      /**
+       * <code>SPLIT_REVERTED = 9;</code>
+       */
+      public static final int SPLIT_REVERTED_VALUE = 9;
+      /**
+       * <code>MERGE_REVERTED = 10;</code>
+       */
+      public static final int MERGE_REVERTED_VALUE = 10;
+
+
+      public final int getNumber() { return value; }
+
+      public static TransitionCode valueOf(int value) {
+        switch (value) {
+          case 0: return OPENED;
+          case 1: return FAILED_OPEN;
+          case 2: return CLOSED;
+          case 3: return READY_TO_SPLIT;
+          case 4: return READY_TO_MERGE;
+          case 5: return SPLIT_PONR;
+          case 6: return MERGE_PONR;
+          case 7: return SPLIT;
+          case 8: return MERGED;
+          case 9: return SPLIT_REVERTED;
+          case 10: return MERGE_REVERTED;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<TransitionCode>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<TransitionCode>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<TransitionCode>() {
+              public TransitionCode findValueByNumber(int number) {
+                return TransitionCode.valueOf(number);
+              }
+            };
+
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final TransitionCode[] VALUES = values();
+
+      public static TransitionCode valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+
+      private final int index;
+      private final int value;
+
+      private TransitionCode(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:RegionTransition.TransitionCode)
+    }
+
+    private int bitField0_;
+    // required .RegionTransition.TransitionCode transition_code = 1;
+    public static final int TRANSITION_CODE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode transitionCode_;
+    /**
+     * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+     */
+    public boolean hasTransitionCode() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode getTransitionCode() {
+      return transitionCode_;
+    }
+
+    // repeated .RegionInfo region_info = 2;
+    public static final int REGION_INFO_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    public int getRegionInfoCount() {
+      return regionInfo_.size();
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+      return regionInfo_.get(index);
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 2;</code>
+     *
+     * <pre>
+     ** Mutliple regions are involved during merging/splitting 
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index) {
+      return regionInfo_.get(index);
+    }
+
+    // optional uint64 open_seq_num = 3;
+    public static final int OPEN_SEQ_NUM_FIELD_NUMBER = 3;
+    private long openSeqNum_;
+    /**
+     * <code>optional uint64 open_seq_num = 3;</code>
+     *
+     * <pre>
+     ** For newly opened region, the open seq num is needed 
+     * </pre>
+     */
+    public boolean hasOpenSeqNum() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional uint64 open_seq_num = 3;</code>
+     *
+     * <pre>
+     ** For newly opened region, the open seq num is needed 
+     * </pre>
+     */
+    public long getOpenSeqNum() {
+      return openSeqNum_;
+    }
+
+    private void initFields() {
+      transitionCode_ = org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode.OPENED;
+      regionInfo_ = java.util.Collections.emptyList();
+      openSeqNum_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasTransitionCode()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getRegionInfoCount(); i++) {
+        if (!getRegionInfo(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, transitionCode_.getNumber());
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        output.writeMessage(2, regionInfo_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(3, openSeqNum_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, transitionCode_.getNumber());
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, regionInfo_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, openSeqNum_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition other = (org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition) obj;
+
+      boolean result = true;
+      result = result && (hasTransitionCode() == other.hasTransitionCode());
+      if (hasTransitionCode()) {
+        result = result &&
+            (getTransitionCode() == other.getTransitionCode());
+      }
+      result = result && getRegionInfoList()
+          .equals(other.getRegionInfoList());
+      result = result && (hasOpenSeqNum() == other.hasOpenSeqNum());
+      if (hasOpenSeqNum()) {
+        result = result && (getOpenSeqNum()
+            == other.getOpenSeqNum());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTransitionCode()) {
+        hash = (37 * hash) + TRANSITION_CODE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getTransitionCode());
+      }
+      if (getRegionInfoCount() > 0) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfoList().hashCode();
+      }
+      if (hasOpenSeqNum()) {
+        hash = (37 * hash) + OPEN_SEQ_NUM_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getOpenSeqNum());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code RegionTransition}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_RegionTransition_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_RegionTransition_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.class, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getRegionInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        transitionCode_ = org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode.OPENED;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        openSeqNum_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_RegionTransition_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition build() {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition result = new org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.transitionCode_ = transitionCode_;
+        if (regionInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.regionInfo_ = regionInfo_;
+        } else {
+          result.regionInfo_ = regionInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.openSeqNum_ = openSeqNum_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.getDefaultInstance()) return this;
+        if (other.hasTransitionCode()) {
+          setTransitionCode(other.getTransitionCode());
+        }
+        if (regionInfoBuilder_ == null) {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfo_.isEmpty()) {
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureRegionInfoIsMutable();
+              regionInfo_.addAll(other.regionInfo_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfoBuilder_.isEmpty()) {
+              regionInfoBuilder_.dispose();
+              regionInfoBuilder_ = null;
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              regionInfoBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getRegionInfoFieldBuilder() : null;
+            } else {
+              regionInfoBuilder_.addAllMessages(other.regionInfo_);
+            }
+          }
+        }
+        if (other.hasOpenSeqNum()) {
+          setOpenSeqNum(other.getOpenSeqNum());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasTransitionCode()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getRegionInfoCount(); i++) {
+          if (!getRegionInfo(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .RegionTransition.TransitionCode transition_code = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode transitionCode_ = org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode.OPENED;
+      /**
+       * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+       */
+      public boolean hasTransitionCode() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode getTransitionCode() {
+        return transitionCode_;
+      }
+      /**
+       * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+       */
+      public Builder setTransitionCode(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        transitionCode_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .RegionTransition.TransitionCode transition_code = 1;</code>
+       */
+      public Builder clearTransitionCode() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        transitionCode_ = org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.TransitionCode.OPENED;
+        onChanged();
+        return this;
+      }
+
+      // repeated .RegionInfo region_info = 2;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+        java.util.Collections.emptyList();
+      private void ensureRegionInfoIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+        if (regionInfoBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        } else {
+          return regionInfoBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public int getRegionInfoCount() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.size();
+        } else {
+          return regionInfoBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);
+        } else {
+          return regionInfoBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder addRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder addRegionInfo(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder addAllRegionInfo(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> values) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          super.addAll(values, regionInfo_);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public Builder removeRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.remove(index);
+          onChanged();
+        } else {
+          regionInfoBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+          int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);  } else {
+          return regionInfoBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+           getRegionInfoOrBuilderList() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+        return getRegionInfoFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 2;</code>
+       *
+       * <pre>
+       ** Mutliple regions are involved during merging/splitting 
+       * </pre>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder> 
+           getRegionInfoBuilderList() {
+        return getRegionInfoFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  regionInfo_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          regionInfo_ = null;
+        }
+        return regionInfoBuilder_;
+      }
+
+      // optional uint64 open_seq_num = 3;
+      private long openSeqNum_ ;
+      /**
+       * <code>optional uint64 open_seq_num = 3;</code>
+       *
+       * <pre>
+       ** For newly opened region, the open seq num is needed 
+       * </pre>
+       */
+      public boolean hasOpenSeqNum() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional uint64 open_seq_num = 3;</code>
+       *
+       * <pre>
+       ** For newly opened region, the open seq num is needed 
+       * </pre>
+       */
+      public long getOpenSeqNum() {
+        return openSeqNum_;
+      }
+      /**
+       * <code>optional uint64 open_seq_num = 3;</code>
+       *
+       * <pre>
+       ** For newly opened region, the open seq num is needed 
+       * </pre>
+       */
+      public Builder setOpenSeqNum(long value) {
+        bitField0_ |= 0x00000004;
+        openSeqNum_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 open_seq_num = 3;</code>
+       *
+       * <pre>
+       ** For newly opened region, the open seq num is needed 
+       * </pre>
+       */
+      public Builder clearOpenSeqNum() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        openSeqNum_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:RegionTransition)
+    }
+
+    static {
+      defaultInstance = new RegionTransition(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:RegionTransition)
+  }
+
+  public interface ReportRegionTransitionRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .ServerName server = 1;
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    boolean hasServer();
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer();
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder();
+
+    // repeated .RegionTransition transition = 2;
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> 
+        getTransitionList();
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition getTransition(int index);
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    int getTransitionCount();
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder> 
+        getTransitionOrBuilderList();
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder getTransitionOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code ReportRegionTransitionRequest}
+   */
+  public static final class ReportRegionTransitionRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements ReportRegionTransitionRequestOrBuilder {
+    // Use ReportRegionTransitionRequest.newBuilder() to construct.
+    private ReportRegionTransitionRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ReportRegionTransitionRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ReportRegionTransitionRequest defaultInstance;
+    public static ReportRegionTransitionRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ReportRegionTransitionRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ReportRegionTransitionRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = server_.toBuilder();
+              }
+              server_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(server_);
+                server_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                transition_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              transition_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          transition_ = java.util.Collections.unmodifiableList(transition_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_ReportRegionTransitionRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_ReportRegionTransitionRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.class, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ReportRegionTransitionRequest> PARSER =
+        new com.google.protobuf.AbstractParser<ReportRegionTransitionRequest>() {
+      public ReportRegionTransitionRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ReportRegionTransitionRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ReportRegionTransitionRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .ServerName server = 1;
+    public static final int SERVER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_;
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    public boolean hasServer() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() {
+      return server_;
+    }
+    /**
+     * <code>required .ServerName server = 1;</code>
+     *
+     * <pre>
+     ** This region server's server name 
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() {
+      return server_;
+    }
+
+    // repeated .RegionTransition transition = 2;
+    public static final int TRANSITION_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> transition_;
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> getTransitionList() {
+      return transition_;
+    }
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder> 
+        getTransitionOrBuilderList() {
+      return transition_;
+    }
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    public int getTransitionCount() {
+      return transition_.size();
+    }
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition getTransition(int index) {
+      return transition_.get(index);
+    }
+    /**
+     * <code>repeated .RegionTransition transition = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder getTransitionOrBuilder(
+        int index) {
+      return transition_.get(index);
+    }
+
+    private void initFields() {
+      server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      transition_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasServer()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getServer().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getTransitionCount(); i++) {
+        if (!getTransition(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, server_);
+      }
+      for (int i = 0; i < transition_.size(); i++) {
+        output.writeMessage(2, transition_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, server_);
+      }
+      for (int i = 0; i < transition_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, transition_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest other = (org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest) obj;
+
+      boolean result = true;
+      result = result && (hasServer() == other.hasServer());
+      if (hasServer()) {
+        result = result && getServer()
+            .equals(other.getServer());
+      }
+      result = result && getTransitionList()
+          .equals(other.getTransitionList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasServer()) {
+        hash = (37 * hash) + SERVER_FIELD_NUMBER;
+        hash = (53 * hash) + getServer().hashCode();
+      }
+      if (getTransitionCount() > 0) {
+        hash = (37 * hash) + TRANSITION_FIELD_NUMBER;
+        hash = (53 * hash) + getTransitionList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ReportRegionTransitionRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_ReportRegionTransitionRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_ReportRegionTransitionRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.class, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getServerFieldBuilder();
+          getTransitionFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (serverBuilder_ == null) {
+          server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+        } else {
+          serverBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (transitionBuilder_ == null) {
+          transition_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          transitionBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.internal_static_ReportRegionTransitionRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest result = new org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (serverBuilder_ == null) {
+          result.server_ = server_;
+        } else {
+          result.server_ = serverBuilder_.build();
+        }
+        if (transitionBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            transition_ = java.util.Collections.unmodifiableList(transition_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.transition_ = transition_;
+        } else {
+          result.transition_ = transitionBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest.getDefaultInstance()) return this;
+        if (other.hasServer()) {
+          mergeServer(other.getServer());
+        }
+        if (transitionBuilder_ == null) {
+          if (!other.transition_.isEmpty()) {
+            if (transition_.isEmpty()) {
+              transition_ = other.transition_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureTransitionIsMutable();
+              transition_.addAll(other.transition_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.transition_.isEmpty()) {
+            if (transitionBuilder_.isEmpty()) {
+              transitionBuilder_.dispose();
+              transitionBuilder_ = null;
+              transition_ = other.transition_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              transitionBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTransitionFieldBuilder() : null;
+            } else {
+              transitionBuilder_.addAllMessages(other.transition_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasServer()) {
+          
+          return false;
+        }
+        if (!getServer().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getTransitionCount(); i++) {
+          if (!getTransition(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionTransitionRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .ServerName server = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public boolean hasServer() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() {
+        if (serverBuilder_ == null) {
+          return server_;
+        } else {
+          return serverBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public Builder setServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          server_ = value;
+          onChanged();
+        } else {
+          serverBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public Builder setServer(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
+        if (serverBuilder_ == null) {
+          server_ = builderForValue.build();
+          onChanged();
+        } else {
+          serverBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public Builder mergeServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              server_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
+            server_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial();
+          } else {
+            server_ = value;
+          }
+          onChanged();
+        } else {
+          serverBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public Builder clearServer() {
+        if (serverBuilder_ == null) {
+          server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+          onChanged();
+        } else {
+          serverBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getServerFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() {
+        if (serverBuilder_ != null) {
+          return serverBuilder_.getMessageOrBuilder();
+        } else {
+          return server_;
+        }
+      }
+      /**
+       * <code>required .ServerName server = 1;</code>
+       *
+       * <pre>
+       ** This region server's server name 
+       * </pre>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
+          getServerFieldBuilder() {
+        if (serverBuilder_ == null) {
+          serverBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
+                  server_,
+                  getParentForChildren(),
+                  isClean());
+          server_ = null;
+        }
+        return serverBuilder_;
+      }
+
+      // repeated .RegionTransition transition = 2;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> transition_ =
+        java.util.Collections.emptyList();
+      private void ensureTransitionIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          transition_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition>(transition_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder> transitionBuilder_;
+
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> getTransitionList() {
+        if (transitionBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(transition_);
+        } else {
+          return transitionBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public int getTransitionCount() {
+        if (transitionBuilder_ == null) {
+          return transition_.size();
+        } else {
+          return transitionBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition getTransition(int index) {
+        if (transitionBuilder_ == null) {
+          return transition_.get(index);
+        } else {
+          return transitionBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder setTransition(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition value) {
+        if (transitionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTransitionIsMutable();
+          transition_.set(index, value);
+          onChanged();
+        } else {
+          transitionBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder setTransition(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder builderForValue) {
+        if (transitionBuilder_ == null) {
+          ensureTransitionIsMutable();
+          transition_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          transitionBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder addTransition(org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition value) {
+        if (transitionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTransitionIsMutable();
+          transition_.add(value);
+          onChanged();
+        } else {
+          transitionBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder addTransition(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition value) {
+        if (transitionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTransitionIsMutable();
+          transition_.add(index, value);
+          onChanged();
+        } else {
+          transitionBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder addTransition(
+          org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder builderForValue) {
+        if (transitionBuilder_ == null) {
+          ensureTransitionIsMutable();
+          transition_.add(builderForValue.build());
+          onChanged();
+        } else {
+          transitionBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder addTransition(
+          int index, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder builderForValue) {
+        if (transitionBuilder_ == null) {
+          ensureTransitionIsMutable();
+          transition_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          transitionBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder addAllTransition(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition> values) {
+        if (transitionBuilder_ == null) {
+          ensureTransitionIsMutable();
+          super.addAll(values, transition_);
+          onChanged();
+        } else {
+          transitionBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder clearTransition() {
+        if (transitionBuilder_ == null) {
+          transition_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          transitionBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public Builder removeTransition(int index) {
+        if (transitionBuilder_ == null) {
+          ensureTransitionIsMutable();
+          transition_.remove(index);
+          onChanged();
+        } else {
+          transitionBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder getTransitionBuilder(
+          int index) {
+        return getTransitionFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder getTransitionOrBuilder(
+          int index) {
+        if (transitionBuilder_ == null) {
+          return transition_.get(index);  } else {
+          return transitionBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder> 
+           getTransitionOrBuilderList() {
+        if (transitionBuilder_ != null) {
+          return transitionBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(transition_);
+        }
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder addTransitionBuilder() {
+        return getTransitionFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder addTransitionBuilder(
+          int index) {
+        return getTransitionFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionTransition transition = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder> 
+           getTransitionBuilderList() {
+        return getTransitionFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder> 
+          getTransitionFieldBuilder() {
+        if (transitionBuilder_ == null) {
+          transitionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransition.Builder, org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionTransitionOrBuilder>(
+                  transition_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          transition_ = null;
+        }
+        return transitionBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ReportRegionTransitionRequest)
+    }
+
+    static {
+      defaultInstance = new ReportRegionTransitionRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ReportRegionTransitionRequest)
+  }
+
+  public interface ReportRegionTransitionResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string error_message = 1;
+    /**
+     * <code>optional string error_message = 1;</code>
+     *
+     * <pre>
+     ** Error message if failed to update the region state 
+     * </pre>
+     */
+    boolean hasErrorMessage();
+    /**
+     * <code>optional string error_message = 1;</code>
+     *
+     * <pre>
+     ** Error message if failed to update the region state 
+     * </pre>
+     */
+    java.lang.String getErrorMessage();
+    /**
+     * <code>optional string error_message = 1;</code>
+     *
+     * <pre>
+     ** Error message if failed to update the region state 
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getErrorMessageBytes();
+  }
+  /**
+   * Protobuf type {@code ReportRegionTransitionResponse}
+   */
+  public static final class ReportRegionTransitionResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements ReportRegionTransitionResponseOrBuilder {
+    // Use ReportRegionTransitionResponse.newBuilder() to construct.
+    private ReportRegionTransitionResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(bui

<TRUNCATED>