You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2014/08/07 01:22:45 UTC

[01/10] HBASE-11611 Clean up ZK-based region assignment

Repository: hbase
Updated Branches:
  refs/heads/master e17a3ca09 -> 17dff6818


http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index 2990014..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/**
- * 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.handler;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.coordination.ZkCloseRegionCoordination;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MockServer;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-/**
- * Test of the {@link CloseRegionHandler}.
- */
-@Category(MediumTests.class)
-public class TestCloseRegionHandler {
-  static final Log LOG = LogFactory.getLog(TestCloseRegionHandler.class);
-  private final static HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU();
-  private static final HTableDescriptor TEST_HTD =
-    new HTableDescriptor(TableName.valueOf("TestCloseRegionHandler"));
-  private HRegionInfo TEST_HRI;
-  private int testIndex = 0;
-
-  @BeforeClass public static void before() throws Exception {
-    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
-    HTU.startMiniZKCluster();
-  }
-
-  @AfterClass public static void after() throws IOException {
-    HTU.shutdownMiniZKCluster();
-  }
-
-  /**
-   * Before each test, use a different HRI, so the different tests
-   * don't interfere with each other. This allows us to use just
-   * a single ZK cluster for the whole suite.
-   */
-  @Before
-  public void setupHRI() {
-    TEST_HRI = new HRegionInfo(TEST_HTD.getTableName(),
-      Bytes.toBytes(testIndex),
-      Bytes.toBytes(testIndex + 1));
-    testIndex++;
-  }
-
-  /**
-   * Test that if we fail a flush, abort gets set on close.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-4270">HBASE-4270</a>
-   * @throws IOException
-   * @throws NodeExistsException
-   * @throws KeeperException
-   */
-  @Test public void testFailedFlushAborts()
-  throws IOException, NodeExistsException, KeeperException {
-    final Server server = new MockServer(HTU, false);
-    final RegionServerServices rss = HTU.createMockRegionServerService();
-    HTableDescriptor htd = TEST_HTD;
-    final HRegionInfo hri =
-      new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
-        HConstants.EMPTY_END_ROW);
-    HRegion region = HTU.createLocalHRegion(hri,  htd);
-    try {
-      assertNotNull(region);
-      // Spy on the region so can throw exception when close is called.
-      HRegion spy = Mockito.spy(region);
-      final boolean abort = false;
-      Mockito.when(spy.close(abort)).
-      thenThrow(new IOException("Mocked failed close!"));
-      // The CloseRegionHandler will try to get an HRegion that corresponds
-      // to the passed hri -- so insert the region into the online region Set.
-      rss.addToOnlineRegions(spy);
-      // Assert the Server is NOT stopped before we call close region.
-      assertFalse(server.isStopped());
-
-      ZkCoordinatedStateManager consensusProvider = new ZkCoordinatedStateManager();
-      consensusProvider.initialize(server);
-      consensusProvider.start();
-
-      ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
-        new ZkCloseRegionCoordination.ZkCloseRegionDetails();
-      zkCrd.setPublishStatusInZk(false);
-      zkCrd.setExpectedVersion(-1);
-
-      CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
-            consensusProvider.getCloseRegionCoordination(), zkCrd);
-      boolean throwable = false;
-      try {
-        handler.process();
-      } catch (Throwable t) {
-        throwable = true;
-      } finally {
-        assertTrue(throwable);
-        // Abort calls stop so stopped flag should be set.
-        assertTrue(server.isStopped());
-      }
-    } finally {
-      HRegion.closeHRegion(region);
-    }
-  }
-
-     /**
-      * Test if close region can handle ZK closing node version mismatch
-      * @throws IOException
-      * @throws NodeExistsException
-      * @throws KeeperException
-     * @throws DeserializationException
-      */
-     @Test public void testZKClosingNodeVersionMismatch()
-     throws IOException, NodeExistsException, KeeperException, DeserializationException {
-       final Server server = new MockServer(HTU);
-       final RegionServerServices rss = HTU.createMockRegionServerService();
-
-       HTableDescriptor htd = TEST_HTD;
-       final HRegionInfo hri = TEST_HRI;
-
-       ZkCoordinatedStateManager coordinationProvider = new ZkCoordinatedStateManager();
-       coordinationProvider.initialize(server);
-       coordinationProvider.start();
-
-       // open a region first so that it can be closed later
-       OpenRegion(server, rss, htd, hri, coordinationProvider.getOpenRegionCoordination());
-
-       // close the region
-       // Create it CLOSING, which is what Master set before sending CLOSE RPC
-       int versionOfClosingNode = ZKAssign.createNodeClosing(server.getZooKeeper(),
-         hri, server.getServerName());
-
-       // The CloseRegionHandler will validate the expected version
-       // Given it is set to invalid versionOfClosingNode+1,
-       // CloseRegionHandler should be M_ZK_REGION_CLOSING
-
-       ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
-         new ZkCloseRegionCoordination.ZkCloseRegionDetails();
-       zkCrd.setPublishStatusInZk(true);
-       zkCrd.setExpectedVersion(versionOfClosingNode+1);
-
-       CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
-         coordinationProvider.getCloseRegionCoordination(), zkCrd);
-       handler.process();
-
-       // Handler should remain in M_ZK_REGION_CLOSING
-       RegionTransition rt =
-         RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
-       assertTrue(rt.getEventType().equals(EventType.M_ZK_REGION_CLOSING ));
-     }
-
-     /**
-      * Test if the region can be closed properly
-      * @throws IOException
-      * @throws NodeExistsException
-      * @throws KeeperException
-     * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
-      */
-     @Test public void testCloseRegion()
-     throws IOException, NodeExistsException, KeeperException, DeserializationException {
-       final Server server = new MockServer(HTU);
-       final RegionServerServices rss = HTU.createMockRegionServerService();
-
-       HTableDescriptor htd = TEST_HTD;
-       HRegionInfo hri = TEST_HRI;
-
-       ZkCoordinatedStateManager coordinationProvider = new ZkCoordinatedStateManager();
-       coordinationProvider.initialize(server);
-       coordinationProvider.start();
-
-       // open a region first so that it can be closed later
-       OpenRegion(server, rss, htd, hri, coordinationProvider.getOpenRegionCoordination());
-
-       // close the region
-       // Create it CLOSING, which is what Master set before sending CLOSE RPC
-       int versionOfClosingNode = ZKAssign.createNodeClosing(server.getZooKeeper(),
-         hri, server.getServerName());
-
-       // The CloseRegionHandler will validate the expected version
-       // Given it is set to correct versionOfClosingNode,
-       // CloseRegionHandlerit should be RS_ZK_REGION_CLOSED
-
-       ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
-         new ZkCloseRegionCoordination.ZkCloseRegionDetails();
-       zkCrd.setPublishStatusInZk(true);
-       zkCrd.setExpectedVersion(versionOfClosingNode);
-
-       CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
-         coordinationProvider.getCloseRegionCoordination(), zkCrd);
-       handler.process();
-       // Handler should have transitioned it to RS_ZK_REGION_CLOSED
-       RegionTransition rt = RegionTransition.parseFrom(
-         ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
-       assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_CLOSED));
-     }
-
-     private void OpenRegion(Server server, RegionServerServices rss,
-         HTableDescriptor htd, HRegionInfo hri, OpenRegionCoordination coordination)
-     throws IOException, NodeExistsException, KeeperException, DeserializationException {
-       // Create it OFFLINE node, which is what Master set before sending OPEN RPC
-       ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
-
-       OpenRegionCoordination.OpenRegionDetails ord =
-         coordination.getDetailsForNonCoordinatedOpening();
-       OpenRegionHandler openHandler =
-         new OpenRegionHandler(server, rss, hri, htd, coordination, ord);
-       rss.getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
-       openHandler.process();
-       // This parse is not used?
-       RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
-       // delete the node, which is what Master do after the region is opened
-       ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
-         EventType.RS_ZK_REGION_OPENED, server.getServerName());
-     }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index d472d57..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
+++ /dev/null
@@ -1,360 +0,0 @@
-/**
- *
- * 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.handler;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
-import org.apache.hadoop.hbase.executor.EventType;
-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.MockServer;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test of the {@link OpenRegionHandler}.
- */
-@Category(MediumTests.class)
-public class TestOpenRegionHandler {
-  static final Log LOG = LogFactory.getLog(TestOpenRegionHandler.class);
-  private final static HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU();
-  private static HTableDescriptor TEST_HTD;
-  private HRegionInfo TEST_HRI;
-
-  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"));
-  }
-
-  @AfterClass public static void after() throws IOException {
-    TEST_HTD = null;
-    HTU.shutdownMiniZKCluster();
-  }
-
-  /**
-   * Before each test, use a different HRI, so the different tests
-   * don't interfere with each other. This allows us to use just
-   * a single ZK cluster for the whole suite.
-   */
-  @Before
-  public void setupHRI() {
-    TEST_HRI = new HRegionInfo(TEST_HTD.getTableName(),
-      Bytes.toBytes(testIndex),
-      Bytes.toBytes(testIndex + 1));
-    testIndex++;
-  }
-
-  /**
-   * Test the openregionhandler can deal with its znode being yanked out from
-   * under it.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-3627">HBASE-3627</a>
-   * @throws IOException
-   * @throws NodeExistsException
-   * @throws KeeperException
-   */
-  @Test public void testYankingRegionFromUnderIt()
-  throws IOException, NodeExistsException, KeeperException {
-    final Server server = new MockServer(HTU);
-    final RegionServerServices rss = HTU.createMockRegionServerService();
-
-    HTableDescriptor htd = TEST_HTD;
-    final HRegionInfo hri = TEST_HRI;
-    HRegion region =
-         HRegion.createHRegion(hri, HTU.getDataTestDir(), HTU
-            .getConfiguration(), htd);
-    assertNotNull(region);
-    try {
-      ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-      csm.initialize(server);
-      csm.start();
-
-      ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-        new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-      zkCrd.setServerName(server.getServerName());
-
-      OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri,
-        htd, csm.getOpenRegionCoordination(), zkCrd) {
-        HRegion openRegion() {
-          // Open region first, then remove znode as though it'd been hijacked.
-          HRegion region = super.openRegion();
-
-          // Don't actually open region BUT remove the znode as though it'd
-          // been hijacked on us.
-          ZooKeeperWatcher zkw = this.server.getZooKeeper();
-          String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-          try {
-            ZKUtil.deleteNodeFailSilent(zkw, node);
-          } catch (KeeperException e) {
-            throw new RuntimeException("Ugh failed delete of " + node, e);
-          }
-          return region;
-        }
-      };
-      rss.getRegionsInTransitionInRS().put(
-        hri.getEncodedNameAsBytes(), Boolean.TRUE);
-      // Call process without first creating OFFLINE region in zk, see if
-      // exception or just quiet return (expected).
-      handler.process();
-      rss.getRegionsInTransitionInRS().put(
-        hri.getEncodedNameAsBytes(), Boolean.TRUE);
-      ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
-      // Call process again but this time yank the zk znode out from under it
-      // post OPENING; again will expect it to come back w/o NPE or exception.
-      handler.process();
-    } finally {
-      HRegion.closeHRegion(region);
-    }
-  }
-  
-  /**
-   * Test the openregionhandler can deal with perceived failure of transitioning to OPENED state
-   * due to intermittent zookeeper malfunctioning.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-9387">HBASE-9387</a>
-   * @throws IOException
-   * @throws NodeExistsException
-   * @throws KeeperException
-   */
-  @Test
-  public void testRegionServerAbortionDueToFailureTransitioningToOpened()
-      throws IOException, NodeExistsException, KeeperException {
-    final Server server = new MockServer(HTU);
-    final RegionServerServices rss = HTU.createMockRegionServerService();
-
-    HTableDescriptor htd = TEST_HTD;
-    final HRegionInfo hri = TEST_HRI;
-    HRegion region =
-         HRegion.createHRegion(hri, HTU.getDataTestDir(), HTU
-            .getConfiguration(), htd);
-    assertNotNull(region);
-    try {
-
-      ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-      csm.initialize(server);
-      csm.start();
-
-      ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-        new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-      zkCrd.setServerName(server.getServerName());
-
-      ZkOpenRegionCoordination openRegionCoordination =
-        new ZkOpenRegionCoordination(csm, server.getZooKeeper()) {
-        @Override
-        public boolean transitionToOpened(final HRegion r, OpenRegionDetails ord)
-            throws IOException {
-          // remove znode simulating intermittent zookeeper connection issue
-          ZooKeeperWatcher zkw = server.getZooKeeper();
-          String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-          try {
-            ZKUtil.deleteNodeFailSilent(zkw, node);
-          } catch (KeeperException e) {
-            throw new RuntimeException("Ugh failed delete of " + node, e);
-          }
-          // then try to transition to OPENED
-          return super.transitionToOpened(r, ord);
-        }
-      };
-
-      OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri, htd,
-        openRegionCoordination, zkCrd);
-      rss.getRegionsInTransitionInRS().put(
-        hri.getEncodedNameAsBytes(), Boolean.TRUE);
-      // Call process without first creating OFFLINE region in zk, see if
-      // exception or just quiet return (expected).
-      handler.process();
-      rss.getRegionsInTransitionInRS().put(
-        hri.getEncodedNameAsBytes(), Boolean.TRUE);
-      ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
-      // Call process again but this time yank the zk znode out from under it
-      // post OPENING; again will expect it to come back w/o NPE or exception.
-      handler.process();
-    } catch (IOException ioe) {
-    } finally {
-      HRegion.closeHRegion(region);
-    }
-    // Region server is expected to abort due to OpenRegionHandler perceiving transitioning
-    // to OPENED as failed
-    // This was corresponding to the second handler.process() call above.
-    assertTrue("region server should have aborted", server.isAborted());
-  }
-  
-  @Test
-  public void testFailedOpenRegion() throws Exception {
-    Server server = new MockServer(HTU);
-    RegionServerServices rsServices = HTU.createMockRegionServerService();
-
-    // Create it OFFLINE, which is what it expects
-    ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
-
-    ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(server);
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(server.getServerName());
-
-    // Create the handler
-    OpenRegionHandler handler =
-      new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
-        csm.getOpenRegionCoordination(), zkCrd) {
-        @Override
-        HRegion openRegion() {
-          // Fake failure of opening a region due to an IOE, which is caught
-          return null;
-        }
-    };
-    rsServices.getRegionsInTransitionInRS().put(
-      TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
-    handler.process();
-
-    // Handler should have transitioned it to FAILED_OPEN
-    RegionTransition rt = RegionTransition.parseFrom(
-      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
-  }
-  
-  @Test
-  public void testFailedUpdateMeta() throws Exception {
-    Server server = new MockServer(HTU);
-    RegionServerServices rsServices = HTU.createMockRegionServerService();
-
-    // Create it OFFLINE, which is what it expects
-    ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
-
-    // Create the handler
-    ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(server);
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(server.getServerName());
-
-    OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
-      csm.getOpenRegionCoordination(), zkCrd) {
-        @Override
-        boolean updateMeta(final HRegion r) {
-          // Fake failure of updating META
-          return false;
-        }
-    };
-    rsServices.getRegionsInTransitionInRS().put(
-      TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
-    handler.process();
-
-    // Handler should have transitioned it to FAILED_OPEN
-    RegionTransition rt = RegionTransition.parseFrom(
-      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
-  }
-  
-  @Test
-  public void testTransitionToFailedOpenEvenIfCleanupFails() throws Exception {
-    Server server = new MockServer(HTU);
-    RegionServerServices rsServices = HTU.createMockRegionServerService();
-    // Create it OFFLINE, which is what it expects
-    ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
-    // Create the handler
-    ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(server);
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(server.getServerName());
-
-    OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
-      csm.getOpenRegionCoordination(), zkCrd) {
-      @Override
-      boolean updateMeta(HRegion r) {
-        return false;
-      };
-
-      @Override
-      void cleanupFailedOpen(HRegion region) throws IOException {
-        throw new IOException("FileSystem got closed.");
-      }
-    };
-    rsServices.getRegionsInTransitionInRS().put(TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
-    try {
-      handler.process();
-    } catch (Exception e) {
-      // Ignore the IOException that we have thrown from cleanupFailedOpen
-    }
-    RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(),
-        TEST_HRI.getEncodedName()));
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
-  }
-
-  @Test
-  public void testTransitionToFailedOpenFromOffline() throws Exception {
-    Server server = new MockServer(HTU);
-    RegionServerServices rsServices = HTU.createMockRegionServerService(server.getServerName());
-    // Create it OFFLINE, which is what it expects
-    ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
-    // Create the handler
-    ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(server);
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(server.getServerName());
-
-    ZkOpenRegionCoordination openRegionCoordination =
-      new ZkOpenRegionCoordination(csm, server.getZooKeeper()) {
-      @Override
-      public boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
-                                                    OpenRegionDetails ord) {
-        return false;
-      }
-    };
-
-    OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
-      openRegionCoordination, zkCrd);
-    rsServices.getRegionsInTransitionInRS().put(TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
-
-    handler.process();
-
-    RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(),
-        TEST_HRI.getEncodedName()));
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
index 7a075a6..409b618 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
@@ -29,17 +29,15 @@ import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 /**
  * This builds a table, removes info from meta, and then rebuilds meta.
  */
 @Category(MediumTests.class)
 public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
 
+  @SuppressWarnings("deprecation")
   @Test(timeout = 120000)
   public void testMetaRebuild() throws Exception {
     wipeOutMeta();
@@ -68,10 +66,9 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
     TEST_UTIL.startMiniZKCluster();
     TEST_UTIL.restartHBaseCluster(3);
     TEST_UTIL.getHBaseAdmin().enableTable(table);
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-    
+
     LOG.info("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     LOG.info("No more RIT in ZK, now doing final test verification");
 
     // everything is good again.

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
index a3225bd..6ebfcd7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
@@ -24,13 +24,10 @@ import static org.junit.Assert.assertFalse;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -71,10 +68,8 @@ public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore {
     TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
     TEST_UTIL.restartHBaseCluster(3);
 
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-
     LOG.info("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     LOG.info("No more RIT in ZK, now doing final test verification");
     int tries = 60;
     while(TEST_UTIL.getHBaseCluster()

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
index 9a17948..b7114d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
@@ -24,14 +24,11 @@ import static org.junit.Assert.assertFalse;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -80,10 +77,8 @@ public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore {
     TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
     TEST_UTIL.restartHBaseCluster(3);
 
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-
     LOG.info("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     LOG.info("No more RIT in ZK, now doing final test verification");
     int tries = 60;
     while(TEST_UTIL.getHBaseCluster()


[05/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 07313db..77d8167 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
@@ -26,13 +26,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.util.ConfigUtil;
 
 /**
  * Handles closing of a region on a region server.
@@ -41,7 +39,7 @@ import org.apache.hadoop.hbase.util.ConfigUtil;
 public class CloseRegionHandler extends EventHandler {
   // NOTE on priorities shutting down.  There are none for close. There are some
   // for open.  I think that is right.  On shutdown, we want the meta to close
-  // before root and both to close after the user regions have closed.  What
+  // after the user regions have closed.  What
   // about the case where master tells us to shutdown a catalog region and we
   // have a running queue of user regions to close?
   private static final Log LOG = LogFactory.getLog(CloseRegionHandler.class);
@@ -53,9 +51,6 @@ public class CloseRegionHandler extends EventHandler {
   // when we are aborting.
   private final boolean abort;
   private ServerName destination;
-  private CloseRegionCoordination closeRegionCoordination;
-  private CloseRegionCoordination.CloseRegionDetails closeRegionDetails;
-  private final boolean useZKForAssignment;
 
   /**
    * This method used internally by the RegionServer to close out regions.
@@ -63,49 +58,25 @@ public class CloseRegionHandler extends EventHandler {
    * @param rsServices
    * @param regionInfo
    * @param abort If the regionserver is aborting.
-   * @param closeRegionCoordination consensus for closing regions
-   * @param crd object carrying details about region close task.
+   * @param destination
    */
   public CloseRegionHandler(final Server server,
       final RegionServerServices rsServices,
       final HRegionInfo regionInfo, final boolean abort,
-      CloseRegionCoordination closeRegionCoordination,
-      CloseRegionCoordination.CloseRegionDetails crd) {
-    this(server, rsServices,  regionInfo, abort, closeRegionCoordination, crd,
-      EventType.M_RS_CLOSE_REGION, null);
-  }
-
-  public CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices,
-      final HRegionInfo regionInfo, final boolean abort,
-      CloseRegionCoordination closeRegionCoordination,
-      CloseRegionCoordination.CloseRegionDetails crd,
       ServerName destination) {
-    this(server, rsServices, regionInfo, abort, closeRegionCoordination, crd,
+    this(server, rsServices, regionInfo, abort,
       EventType.M_RS_CLOSE_REGION, destination);
   }
 
-  public CloseRegionHandler(final Server server,
+  protected CloseRegionHandler(final Server server,
       final RegionServerServices rsServices, HRegionInfo regionInfo,
-      boolean abort, CloseRegionCoordination closeRegionCoordination,
-      CloseRegionCoordination.CloseRegionDetails crd, EventType eventType) {
-    this(server, rsServices, regionInfo, abort, closeRegionCoordination, crd, eventType, null);
-  }
-
-    protected CloseRegionHandler(final Server server,
-      final RegionServerServices rsServices, HRegionInfo regionInfo,
-      boolean abort, CloseRegionCoordination closeRegionCoordination,
-      CloseRegionCoordination.CloseRegionDetails crd,
-      EventType eventType, ServerName destination) {
+      boolean abort, EventType eventType, ServerName destination) {
     super(server, eventType);
     this.server = server;
     this.rsServices = rsServices;
     this.regionInfo = regionInfo;
     this.abort = abort;
     this.destination = destination;
-    this.closeRegionCoordination = closeRegionCoordination;
-    this.closeRegionDetails = crd;
-    useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
   }
 
   public HRegionInfo getRegionInfo() {
@@ -128,16 +99,8 @@ public class CloseRegionHandler extends EventHandler {
 
       // Close the region
       try {
-        if (useZKForAssignment && closeRegionCoordination.checkClosingState(
-            regionInfo, closeRegionDetails)) {
-          return;
-        }
-
-        // TODO: If we need to keep updating CLOSING stamp to prevent against
-        // a timeout if this is long-running, need to spin up a thread?
         if (region.close(abort) == null) {
-          // This region got closed.  Most likely due to a split. So instead
-          // of doing the setClosedState() below, let's just ignore cont
+          // This region got closed.  Most likely due to a split.
           // The split message will clean up the master state.
           LOG.warn("Can't close region: was already closed during close(): " +
             regionInfo.getRegionNameAsString());
@@ -153,12 +116,7 @@ public class CloseRegionHandler extends EventHandler {
       }
 
       this.rsServices.removeFromOnlineRegions(region, destination);
-      if (!useZKForAssignment) {
-        rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
-      } else {
-        closeRegionCoordination.setClosedState(region, this.server.getServerName(),
-          closeRegionDetails);
-      }
+      rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
 
       // Done!  Region is closed on this RS
       LOG.debug("Closed " + region.getRegionNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
index f2d5f1f..21e84ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
 
 /**
  * Handles opening of a meta region on a region server.
@@ -35,9 +34,7 @@ import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
 public class OpenMetaHandler extends OpenRegionHandler {
   public OpenMetaHandler(final Server server,
       final RegionServerServices rsServices, HRegionInfo regionInfo,
-      final HTableDescriptor htd, OpenRegionCoordination coordination,
-      OpenRegionCoordination.OpenRegionDetails ord) {
-    super(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_META,
-        coordination, ord);
+      final HTableDescriptor htd) {
+    super(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_META);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 ef30f08..20e0970 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
@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@@ -35,7 +34,6 @@ 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;
 /**
  * Handles opening of a region on a region server.
  * <p>
@@ -50,30 +48,19 @@ public class OpenRegionHandler extends EventHandler {
   private final HRegionInfo regionInfo;
   private final HTableDescriptor htd;
 
-  private OpenRegionCoordination coordination;
-  private OpenRegionCoordination.OpenRegionDetails ord;
-
-  private final boolean useZKForAssignment;
-
   public OpenRegionHandler(final Server server,
       final RegionServerServices rsServices, HRegionInfo regionInfo,
-      HTableDescriptor htd, OpenRegionCoordination coordination,
-      OpenRegionCoordination.OpenRegionDetails ord) {
-    this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION,
-        coordination, ord);
+      HTableDescriptor htd) {
+    this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION);
   }
 
   protected OpenRegionHandler(final Server server,
       final RegionServerServices rsServices, final HRegionInfo regionInfo,
-      final HTableDescriptor htd, EventType eventType,
-      OpenRegionCoordination coordination, OpenRegionCoordination.OpenRegionDetails ord) {
+      final HTableDescriptor htd, EventType eventType) {
     super(server, eventType);
     this.rsServices = rsServices;
     this.regionInfo = regionInfo;
     this.htd = htd;
-    this.coordination = coordination;
-    this.ord = ord;
-    useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
   }
 
   public HRegionInfo getRegionInfo() {
@@ -83,7 +70,6 @@ public class OpenRegionHandler extends EventHandler {
   @Override
   public void process() throws IOException {
     boolean openSuccessful = false;
-    boolean transitionedToOpening = false;
     final String regionName = regionInfo.getRegionNameAsString();
     HRegion region = null;
 
@@ -93,10 +79,9 @@ public class OpenRegionHandler extends EventHandler {
       }
       final String encodedName = regionInfo.getEncodedName();
 
-      // 3 different difficult situations can occur
+      // 2 different difficult situations can occur
       // 1) The opening was cancelled. This is an expected situation
-      // 2) The region was hijacked, we no longer have the znode
-      // 3) The region is now marked as online while we're suppose to open. This would be a bug.
+      // 2) The region is now marked as online while we're suppose to open. This would be a bug.
 
       // Check that this region is not already online
       if (this.rsServices.getFromOnlineRegions(encodedName) != null) {
@@ -106,21 +91,13 @@ public class OpenRegionHandler extends EventHandler {
         return;
       }
 
-      // Check that we're still supposed to open the region and transition.
+      // Check that we're still supposed to open the region.
       // If fails, just return.  Someone stole the region from under us.
-      // Calling transitionFromOfflineToOpening initializes this.version.
       if (!isRegionStillOpening()){
         LOG.error("Region " + encodedName + " opening cancelled");
         return;
       }
 
-      if (useZKForAssignment
-          && !coordination.transitionFromOfflineToOpening(regionInfo, ord)) {
-        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;
-      }
-      transitionedToOpening = true;
       // Open region.  After a successful open, failures in subsequent
       // processing needs to do a close as part of cleanup.
       region = openRegion();
@@ -128,37 +105,15 @@ public class OpenRegionHandler extends EventHandler {
         return;
       }
 
-      boolean failed = true;
-      if (!useZKForAssignment ||
-          coordination.tickleOpening(ord, regionInfo, rsServices, "post_region_open")) {
-        if (updateMeta(region)) {
-          failed = false;
-        }
-      }
-      if (failed || this.server.isStopped() ||
+      if (!updateMeta(region) || this.server.isStopped() ||
           this.rsServices.isStopping()) {
         return;
       }
 
-      if (!isRegionStillOpening() ||
-          (useZKForAssignment && !coordination.transitionToOpened(region, ord))) {
-        // 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
-        // OR (c) someone cancelled the open
-        // In all cases, we try to transition to failed_open to be safe.
+      if (!isRegionStillOpening()) {
         return;
       }
 
-      // We have a znode in the opened state now. We can't really delete it as the master job.
-      // Transitioning to failed open would create a race condition if the master has already
-      // acted the transition to opened.
-      // Cancelling the open is dangerous, because we would have a state where the master thinks
-      // the region is opened while the region is actually closed. It is a dangerous state
-      // to be in. For this reason, from now on, we're not going back. There is a message in the
-      // finally close to let the admin knows where we stand.
-
-
       // Successful region open, and add it to OnlineRegions
       this.rsServices.addToOnlineRegions(region);
       openSuccessful = true;
@@ -166,12 +121,10 @@ public class OpenRegionHandler extends EventHandler {
       // Done!  Successful region open
       LOG.debug("Opened " + regionName + " on " +
         this.server.getServerName());
-
-
     } finally {
       // Do all clean up here
       if (!openSuccessful) {
-        doCleanUpOnFailedOpen(region, transitionedToOpening, ord);
+        doCleanUpOnFailedOpen(region);
       }
       final Boolean current = this.rsServices.getRegionsInTransitionInRS().
           remove(this.regionInfo.getEncodedNameAsBytes());
@@ -180,9 +133,7 @@ public class OpenRegionHandler extends EventHandler {
       // A better solution would be to not have any race condition.
       // this.rsServices.getRegionsInTransitionInRS().remove(
       //  this.regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
-      // would help, but we would still have a consistency issue to manage with
-      // 1) this.rsServices.addToOnlineRegions(region);
-      // 2) the ZK state.
+      // would help.
       if (openSuccessful) {
         if (current == null) { // Should NEVER happen, but let's be paranoid.
           LOG.error("Bad state: we've just opened a region that was NOT in transition. Region="
@@ -198,29 +149,14 @@ public class OpenRegionHandler extends EventHandler {
     }
   }
 
-  private void doCleanUpOnFailedOpen(HRegion region, boolean transitionedToOpening,
-                                     OpenRegionCoordination.OpenRegionDetails ord)
+  private void doCleanUpOnFailedOpen(HRegion region)
       throws IOException {
-    if (transitionedToOpening) {
-      try {
-        if (region != null) {
-          cleanupFailedOpen(region);
-        }
-      } finally {
-        if (!useZKForAssignment) {
-          rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo);
-        } else {
-          // Even if cleanupFailed open fails we need to do this transition
-          // See HBASE-7698
-          coordination.tryTransitionFromOpeningToFailedOpen(regionInfo, ord);
-        }
+    try {
+      if (region != null) {
+        cleanupFailedOpen(region);
       }
-    } else if (!useZKForAssignment) {
+    } finally {
       rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo);
-    } else {
-      // If still transition to OPENING is not done, we need to transition znode
-      // to FAILED_OPEN
-      coordination.tryTransitionFromOfflineToFailedOpen(this.rsServices, regionInfo, ord);
     }
   }
 
@@ -244,8 +180,6 @@ public class OpenRegionHandler extends EventHandler {
     // Post open deploy task:
     //   meta => update meta location in ZK
     //   other region => update meta
-    // It could fail if ZK/meta is not available and
-    // the update runs out of retries.
     long now = System.currentTimeMillis();
     long lastUpdate = now;
     boolean tickleOpening = true;
@@ -255,10 +189,6 @@ public class OpenRegionHandler extends EventHandler {
       if (elapsed > 120000) { // 2 minutes, no need to tickleOpening too often
         // Only tickle OPENING if postOpenDeployTasks is taking some time.
         lastUpdate = now;
-        if (useZKForAssignment) {
-          tickleOpening = coordination.tickleOpening(
-            ord, regionInfo, rsServices, "post_open_deploy");
-        }
       }
       synchronized (signaller) {
         try {
@@ -356,11 +286,6 @@ public class OpenRegionHandler extends EventHandler {
         this.rsServices,
         new CancelableProgressable() {
           public boolean progress() {
-            if (useZKForAssignment) {
-              // if tickle failed, we need to cancel opening region.
-              return coordination.tickleOpening(ord, regionInfo,
-                rsServices, "open_region_progress");
-            }
             if (!isRegionStillOpening()) {
               LOG.warn("Open region aborted since it isn't opening any more");
               return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index 0f9b713..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigUtil.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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", false);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index 84ffec8..960d69c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -147,12 +147,13 @@ public class HBaseFsckRepair {
    * Contacts a region server and waits up to hbase.hbck.close.timeout ms
    * (default 120s) to close the region.  This bypasses the active hmaster.
    */
+  @SuppressWarnings("deprecation")
   public static void closeRegionSilentlyAndWait(Admin admin,
       ServerName server, HRegionInfo region) throws IOException, InterruptedException {
     HConnection connection = admin.getConnection();
     AdminService.BlockingInterface rs = connection.getAdmin(server);
     try {
-      ProtobufUtil.closeRegion(rs, server, region.getRegionName(), false);
+      ProtobufUtil.closeRegion(rs, server, region.getRegionName());
     } catch (IOException e) {
       LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/data/TestNamespaceUpgrade.tgz
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/data/TestNamespaceUpgrade.tgz b/hbase-server/src/test/data/TestNamespaceUpgrade.tgz
deleted file mode 100644
index db6e9c3..0000000
Binary files a/hbase-server/src/test/data/TestNamespaceUpgrade.tgz and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index c67f3f8..7aeb778 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -99,7 +99,6 @@ import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -109,8 +108,6 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.TaskLog;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
@@ -1470,6 +1467,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
    */
+  @SuppressWarnings("serial")
   public static void modifyTableSync(Admin admin, HTableDescriptor desc)
       throws IOException, InterruptedException {
     admin.modifyTable(desc.getTableName(), desc);
@@ -3009,30 +3007,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return zkw;
   }
 
-  /**
-   * Creates a znode with OPENED state.
-   * @param TEST_UTIL
-   * @param region
-   * @param serverName
-   * @return
-   * @throws IOException
-   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
-   * @throws KeeperException
-   * @throws NodeExistsException
-   */
-  public static ZooKeeperWatcher createAndForceNodeToOpenedState(
-      HBaseTestingUtility TEST_UTIL, HRegion region,
-      ServerName serverName) throws ZooKeeperConnectionException,
-      IOException, KeeperException, NodeExistsException {
-    ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
-    ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
-    int version = ZKAssign.transitionNodeOpening(zkw, region
-        .getRegionInfo(), serverName);
-    ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
-        version);
-    return zkw;
-  }
-
   public static void assertKVListsEqual(String additionalMsg,
       final List<? extends Cell> expected,
       final List<? extends Cell> actual) {
@@ -3441,6 +3415,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
+   * Wait until no regions in transition.
+   * @param timeout How long to wait.
+   * @throws Exception
+   */
+  public void waitUntilNoRegionsInTransition(
+      final long timeout) throws Exception {
+    waitFor(timeout, predicateNoRegionsInTransition());
+  }
+
+  /**
    * Create a set of column descriptors with the combination of compression,
    * encoding, bloom codecs available.
    * @return the list of column descriptors

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index db087ff..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/**
- * 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;
-
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.executor.ExecutorType;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-
-/**
- * Test the draining servers feature.
- */
-@Category(MediumTests.class)
-public class TestDrainingServer {
-  private static final Log LOG = LogFactory.getLog(TestDrainingServer.class);
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Abortable abortable = new Abortable() {
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-
-    @Override
-    public void abort(String why, Throwable e) {
-    }
-  };
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-    TEST_UTIL.shutdownMiniZKCluster();
-  }
-  
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
-    TEST_UTIL.startMiniZKCluster();
-  }
-
-  @Test
-  public void testAssignmentManagerDoesntUseDrainingServer() throws Exception {
-    AssignmentManager am;
-    Configuration conf = TEST_UTIL.getConfiguration();
-    final HMaster master = Mockito.mock(HMaster.class);
-    final Server server = Mockito.mock(Server.class);
-    final ServerManager serverManager = Mockito.mock(ServerManager.class);
-    final ServerName SERVERNAME_A = ServerName.valueOf("mockserver_a.org", 1000, 8000);
-    final ServerName SERVERNAME_B = ServerName.valueOf("mockserver_b.org", 1001, 8000);
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
-    final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
-        HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-
-    ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-      "zkWatcher-Test", abortable, true);
-
-    Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
-
-    onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
-
-    Mockito.when(server.getConfiguration()).thenReturn(conf);
-    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
-    Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
-
-    CoordinatedStateManager cp = new ZkCoordinatedStateManager();
-    cp.initialize(server);
-    cp.start();
-
-    Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
-
-    Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
-    Mockito.when(serverManager.getOnlineServersList())
-    .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
-    
-    Mockito.when(serverManager.createDestinationServersList())
-        .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
-    Mockito.when(serverManager.createDestinationServersList(null))
-        .thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
-    
-    for (ServerName sn : onlineServers.keySet()) {
-      Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true);
-      Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1)).thenReturn(true);
-      Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1, null, false)).thenReturn(true);
-      Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>()))
-      .thenReturn(RegionOpeningState.OPENED);
-      Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null))
-      .thenReturn(RegionOpeningState.OPENED);
-      Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true);
-    }
-
-    Mockito.when(master.getServerManager()).thenReturn(serverManager);
-
-    am = new AssignmentManager(server, serverManager,
-        balancer, startupMasterExecutor("mockExecutorService"), null, null);
-
-    Mockito.when(master.getAssignmentManager()).thenReturn(am);
-    Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher);
-
-    am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
-
-    zkWatcher.registerListenerFirst(am);
-
-    addServerToDrainedList(SERVERNAME_A, onlineServers, serverManager);
-
-    am.assign(REGIONINFO, true);
-
-    setRegionOpenedOnZK(zkWatcher, SERVERNAME_A, REGIONINFO);
-    setRegionOpenedOnZK(zkWatcher, SERVERNAME_B, REGIONINFO);
-
-    am.waitForAssignment(REGIONINFO);
-
-    assertTrue(am.getRegionStates().isRegionOnline(REGIONINFO));
-    assertNotEquals(am.getRegionStates().getRegionServerOfRegion(REGIONINFO), SERVERNAME_A);
-  }
-
-  @Test
-  public void testAssignmentManagerDoesntUseDrainedServerWithBulkAssign() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
-    AssignmentManager am;
-    final HMaster master = Mockito.mock(HMaster.class);
-    final Server server = Mockito.mock(Server.class);
-    final ServerManager serverManager = Mockito.mock(ServerManager.class);
-    final ServerName SERVERNAME_A = ServerName.valueOf("mockserverbulk_a.org", 1000, 8000);
-    final ServerName SERVERNAME_B = ServerName.valueOf("mockserverbulk_b.org", 1001, 8000);
-    final ServerName SERVERNAME_C = ServerName.valueOf("mockserverbulk_c.org", 1002, 8000);
-    final ServerName SERVERNAME_D = ServerName.valueOf("mockserverbulk_d.org", 1003, 8000);
-    final ServerName SERVERNAME_E = ServerName.valueOf("mockserverbulk_e.org", 1004, 8000);
-    final Map<HRegionInfo, ServerName> bulk = new HashMap<HRegionInfo, ServerName>();
-
-    Set<ServerName> bunchServersAssigned = new HashSet<ServerName>();
-    
-    HRegionInfo REGIONINFO_A = new HRegionInfo(TableName.valueOf("table_A"),
-        HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-    HRegionInfo REGIONINFO_B = new HRegionInfo(TableName.valueOf("table_B"),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-    HRegionInfo REGIONINFO_C = new HRegionInfo(TableName.valueOf("table_C"),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-    HRegionInfo REGIONINFO_D = new HRegionInfo(TableName.valueOf("table_D"),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-    HRegionInfo REGIONINFO_E = new HRegionInfo(TableName.valueOf("table_E"),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-
-    Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
-    List<ServerName> drainedServers = new ArrayList<ServerName>();
-
-    onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_C, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_D, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_E, ServerLoad.EMPTY_SERVERLOAD);
-
-    bulk.put(REGIONINFO_A, SERVERNAME_A);
-    bulk.put(REGIONINFO_B, SERVERNAME_B);
-    bulk.put(REGIONINFO_C, SERVERNAME_C);
-    bulk.put(REGIONINFO_D, SERVERNAME_D);
-    bulk.put(REGIONINFO_E, SERVERNAME_E);
-
-    ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-        "zkWatcher-BulkAssignTest", abortable, true);
-
-    Mockito.when(server.getConfiguration()).thenReturn(conf);
-    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
-    Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
-
-    CoordinatedStateManager cp = new ZkCoordinatedStateManager();
-    cp.initialize(server);
-    cp.start();
-
-    Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
-
-    Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
-    Mockito.when(serverManager.getOnlineServersList()).thenReturn(
-      new ArrayList<ServerName>(onlineServers.keySet()));
-    
-    Mockito.when(serverManager.createDestinationServersList()).thenReturn(
-      new ArrayList<ServerName>(onlineServers.keySet()));
-    Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(
-      new ArrayList<ServerName>(onlineServers.keySet()));
-    
-    for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) {
-      Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true);
-      Mockito.when(serverManager.sendRegionClose(entry.getValue(), 
-        entry.getKey(), -1)).thenReturn(true);
-      Mockito.when(serverManager.sendRegionOpen(entry.getValue(), 
-        entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED);  
-      Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true);
-    }
-    
-    Mockito.when(master.getServerManager()).thenReturn(serverManager);
-
-    drainedServers.add(SERVERNAME_A);
-    drainedServers.add(SERVERNAME_B);
-    drainedServers.add(SERVERNAME_C);
-    drainedServers.add(SERVERNAME_D);
-
-    am = new AssignmentManager(server, serverManager,
-      balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null);
-    
-    Mockito.when(master.getAssignmentManager()).thenReturn(am);
-
-    zkWatcher.registerListener(am);
-    
-    for (ServerName drained : drainedServers) {
-      addServerToDrainedList(drained, onlineServers, serverManager);
-    }
-    
-    am.assign(bulk);
-
-    Map<String, RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
-    for (Entry<String, RegionState> entry : regionsInTransition.entrySet()) {
-      setRegionOpenedOnZK(zkWatcher, entry.getValue().getServerName(), 
-        entry.getValue().getRegion());
-    }
-    
-    am.waitForAssignment(REGIONINFO_A);
-    am.waitForAssignment(REGIONINFO_B);
-    am.waitForAssignment(REGIONINFO_C);
-    am.waitForAssignment(REGIONINFO_D);
-    am.waitForAssignment(REGIONINFO_E);
-    
-    Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments();
-    for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) {
-      LOG.info("Region Assignment: " 
-          + entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue());
-      bunchServersAssigned.add(entry.getValue());
-    }
-    
-    for (ServerName sn : drainedServers) {
-      assertFalse(bunchServersAssigned.contains(sn));
-    }
-  }
-
-  private void addServerToDrainedList(ServerName serverName, 
-      Map<ServerName, ServerLoad> onlineServers, ServerManager serverManager) {
-    onlineServers.remove(serverName);
-    List<ServerName> availableServers = new ArrayList<ServerName>(onlineServers.keySet());
-    Mockito.when(serverManager.createDestinationServersList()).thenReturn(availableServers);
-    Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(availableServers);
-  }
-
-  private void setRegionOpenedOnZK(final ZooKeeperWatcher zkWatcher, final ServerName serverName,
-                                   HRegionInfo hregionInfo) throws Exception {
-    int version = ZKAssign.getVersion(zkWatcher, hregionInfo);
-    int versionTransition = ZKAssign.transitionNode(zkWatcher,
-        hregionInfo, serverName, EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, version);
-    ZKAssign.transitionNodeOpened(zkWatcher, hregionInfo, serverName, versionTransition);
-  }
-
-  private ExecutorService startupMasterExecutor(final String name) {
-    ExecutorService executor = new ExecutorService(name);
-    executor.startExecutorService(ExecutorType.MASTER_OPEN_REGION, 3);
-    executor.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, 3);
-    executor.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, 3);
-    executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
-    return executor;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 1e3e3af..990de0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -244,7 +244,7 @@ public class TestIOFencing {
     c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class);
     c.setBoolean("dfs.support.append", true);
     // Encourage plenty of flushes
-    c.setLong("hbase.hregion.memstore.flush.size", 200000);
+    c.setLong("hbase.hregion.memstore.flush.size", 100000);
     c.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, ConstantSizeRegionSplitPolicy.class.getName());
     // Only run compaction when we tell it to
     c.setInt("hbase.hstore.compaction.min",1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 1448bf2..359bb0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -26,12 +26,12 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.List;
 import java.util.Random;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTable;
@@ -48,6 +48,7 @@ import org.junit.experimental.categories.Category;
  * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
  */
 @Category(MediumTests.class)
+@SuppressWarnings("deprecation")
 public class TestMetaTableAccessor {
   private static final Log LOG = LogFactory.getLog(TestMetaTableAccessor.class);
   private static final  HBaseTestingUtility UTIL = new HBaseTestingUtility();

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index e65430b..9d502d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -500,8 +499,7 @@ public class TestZooKeeper {
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
       admin.createTable(htd, SPLIT_KEYS);
-      ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-      ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
+      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       m.getZooKeeper().close();
       MockLoadBalancer.retainAssignCalled = false;
       m.abort("Test recovery from zk session expired",
@@ -524,8 +522,7 @@ public class TestZooKeeper {
    * RS goes down.
    */
   @Test(timeout = 300000)
-  public void testLogSplittingAfterMasterRecoveryDueToZKExpiry() throws IOException,
-      KeeperException, InterruptedException {
+  public void testLogSplittingAfterMasterRecoveryDueToZKExpiry() throws Exception {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     cluster.startRegionServer();
     HMaster m = cluster.getMaster();
@@ -541,8 +538,7 @@ public class TestZooKeeper {
       HColumnDescriptor hcd = new HColumnDescriptor("col");
       htd.addFamily(hcd);
       admin.createTable(htd, SPLIT_KEYS);
-      ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-      ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
+      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       table = new HTable(TEST_UTIL.getConfiguration(), tableName);
       Put p;
       int numberOfPuts;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index bc3a1f8..02fd1c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -19,6 +19,18 @@
 
 package org.apache.hadoop.hbase.client;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,7 +55,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -53,23 +64,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * Tests for region replicas. Sad that we cannot isolate these without bringing up a whole
  * cluster. See {@link org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster}.
  */
 @Category(MediumTests.class)
+@SuppressWarnings("deprecation")
 public class TestReplicasClient {
   private static final Log LOG = LogFactory.getLog(TestReplicasClient.class);
 
@@ -187,11 +187,13 @@ public class TestReplicasClient {
       } catch (MasterNotRunningException ignored) {
       }
     }
+    ha.close();
     LOG.info("Master has stopped");
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
+    HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
     if (table != null) table.close();
     HTU.shutdownMiniCluster();
   }
@@ -219,8 +221,6 @@ public class TestReplicasClient {
       closeRegion(hriPrimary);
     } catch (Exception ignored) {
     }
-    ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriPrimary);
-    ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriSecondary);
 
     HTU.getHBaseAdmin().getConnection().clearRegionCache();
   }
@@ -233,10 +233,9 @@ public class TestReplicasClient {
     try {
       if (isRegionOpened(hri)) return;
     } catch (Exception e){}
-    ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
     // first version is '0'
     AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
-      getRS().getServerName(), hri, 0, null, null);
+      getRS().getServerName(), hri, null, null);
     AdminProtos.OpenRegionResponse responseOpen = getRS().getRSRpcServices().openRegion(null, orr);
     Assert.assertEquals(responseOpen.getOpeningStateCount(), 1);
     Assert.assertEquals(responseOpen.getOpeningState(0),
@@ -245,27 +244,19 @@ public class TestReplicasClient {
   }
 
   private void closeRegion(HRegionInfo hri) throws Exception {
-    ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-
     AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
-      getRS().getServerName(), hri.getEncodedName(), true);
+      getRS().getServerName(), hri.getEncodedName());
     AdminProtos.CloseRegionResponse responseClose = getRS()
         .getRSRpcServices().closeRegion(null, crr);
     Assert.assertTrue(responseClose.getClosed());
 
     checkRegionIsClosed(hri.getEncodedName());
-
-    ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), null);
   }
 
   private void checkRegionIsOpened(HRegionInfo hri) throws Exception {
-
     while (!getRS().getRegionsInTransitionInRS().isEmpty()) {
       Thread.sleep(1);
     }
-
-    Assert.assertTrue(
-        ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), null));
   }
 
   private boolean isRegionOpened(HRegionInfo hri) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 380b337..a7b3319 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,10 +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;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -474,7 +471,7 @@ public class TestScannersFromClientSide {
     int i = cluster.getServerWith(regionName);
     HRegionServer rs = cluster.getRegionServer(i);
     ProtobufUtil.closeRegion(
-      rs.getRSRpcServices(), rs.getServerName(), regionName, false);
+      rs.getRSRpcServices(), rs.getServerName(), regionName);
     long startTime = EnvironmentEdgeManager.currentTimeMillis();
     long timeOut = 300000;
     while (true) {
@@ -487,27 +484,19 @@ public class TestScannersFromClientSide {
     }
 
     // Now open the region again.
-    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
-    try {
-      HMaster master = cluster.getMaster();
-      RegionStates states = master.getAssignmentManager().getRegionStates();
-      states.regionOffline(hri);
-      states.updateRegionState(hri, State.OPENING);
-      if (ConfigUtil.useZKForAssignment(TEST_UTIL.getConfiguration())) {
-        ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
-      }
-      ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
-      startTime = EnvironmentEdgeManager.currentTimeMillis();
-      while (true) {
-        if (rs.getOnlineRegion(regionName) != null) {
-          break;
-        }
-        assertTrue("Timed out in open the testing region",
-          EnvironmentEdgeManager.currentTimeMillis() < startTime + timeOut);
-        Thread.sleep(500);
+    HMaster master = cluster.getMaster();
+    RegionStates states = master.getAssignmentManager().getRegionStates();
+    states.regionOffline(hri);
+    states.updateRegionState(hri, State.OPENING);
+    ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
+    startTime = EnvironmentEdgeManager.currentTimeMillis();
+    while (true) {
+      if (rs.getOnlineRegion(regionName) != null) {
+        break;
       }
-    } finally {
-      ZKAssign.deleteNodeFailSilent(zkw, hri);
+      assertTrue("Timed out in open the testing region",
+        EnvironmentEdgeManager.currentTimeMillis() < startTime + timeOut);
+      Thread.sleep(500);
     }
 
     // c0:0, c1:1

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
index 0ceb953..2b1f6f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -194,7 +193,7 @@ public class TestChangingEncoding {
     // wait for regions out of transition. Otherwise, for online
     // encoding change, verification phase may be flaky because
     // regions could be still in transition.
-    ZKAssign.blockUntilNoRIT(TEST_UTIL.getZooKeeperWatcher());
+    TEST_UTIL.waitUntilNoRegionsInTransition(TIMEOUT_MS);
   }
 
   @Test(timeout=TIMEOUT_MS)

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java
deleted file mode 100644
index 10127c8..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * 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 static org.junit.Assert.assertNotSame;
-
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Package scoped mocking utility.
- */
-public class Mocking {
-
-  static void waitForRegionFailedToCloseAndSetToPendingClose(
-      AssignmentManager am, HRegionInfo hri) throws InterruptedException {
-    // Since region server is fake, sendRegionClose will fail, and closing
-    // region will fail. For testing purpose, moving it back to pending close
-    boolean wait = true;
-    while (wait) {
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      if (state != null && state.isFailedClose()){
-        am.getRegionStates().updateRegionState(hri, State.PENDING_CLOSE);
-        wait = false;
-      } else {
-        Thread.sleep(1);
-      }
-    }
-  }
-
-  static void waitForRegionPendingOpenInRIT(AssignmentManager am, String encodedName)
-    throws InterruptedException {
-    // We used to do a check like this:
-    //!Mocking.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
-    // There is a race condition with this: because we may do the transition to
-    // RS_ZK_REGION_OPENING before the RIT is internally updated. We need to wait for the
-    // RIT to be as we need it to be instead. This cannot happen in a real cluster as we
-    // update the RIT before sending the openRegion request.
-
-    boolean wait = true;
-    while (wait) {
-      RegionState state = am.getRegionStates()
-        .getRegionsInTransition().get(encodedName);
-      if (state != null && state.isPendingOpen()){
-        wait = false;
-      } else {
-        Thread.sleep(1);
-      }
-    }
-  }
-
-  /**
-   * Verifies that the specified region is in the specified state in ZooKeeper.
-   * <p>
-   * Returns true if region is in transition and in the specified state in
-   * ZooKeeper.  Returns false if the region does not exist in ZK or is in
-   * a different state.
-   * <p>
-   * Method synchronizes() with ZK so will yield an up-to-date result but is
-   * a slow read.
-   * @param zkw
-   * @param region
-   * @param expectedState
-   * @return true if region exists and is in expected state
-   * @throws DeserializationException
-   */
-  static boolean verifyRegionState(ZooKeeperWatcher zkw, HRegionInfo region, EventType expectedState)
-  throws KeeperException, DeserializationException {
-    String encoded = region.getEncodedName();
-
-    String node = ZKAssign.getNodeName(zkw, encoded);
-    zkw.sync(node);
-
-    // Read existing data of the node
-    byte [] existingBytes = null;
-    try {
-      existingBytes = ZKUtil.getDataAndWatch(zkw, node);
-    } catch (KeeperException.NoNodeException nne) {
-      return false;
-    } catch (KeeperException e) {
-      throw e;
-    }
-    if (existingBytes == null) return false;
-    RegionTransition rt = RegionTransition.parseFrom(existingBytes);
-    return rt.getEventType().equals(expectedState);
-  }
-}


[10/10] git commit: HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
HBASE-11611 Clean up ZK-based region assignment


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

Branch: refs/heads/master
Commit: 17dff6818e80305ea55da0519cbcf44de644992e
Parents: e17a3ca
Author: Jimmy Xiang <jx...@cloudera.com>
Authored: Tue Jul 29 16:15:42 2014 -0700
Committer: Jimmy Xiang <jx...@cloudera.com>
Committed: Wed Aug 6 16:22:08 2014 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/RegionTransition.java   |  139 --
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |    4 +-
 .../apache/hadoop/hbase/master/RegionState.java |   36 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   18 +-
 .../hadoop/hbase/protobuf/RequestConverter.java |   47 +-
 .../apache/hadoop/hbase/zookeeper/ZKAssign.java | 1057 -----------
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |    9 +-
 .../hbase/zookeeper/ZooKeeperWatcher.java       |   16 +-
 .../protobuf/generated/ZooKeeperProtos.java     | 1177 +-----------
 .../src/main/protobuf/ZooKeeper.proto           |   15 -
 .../BaseCoordinatedStateManager.java            |   20 -
 .../coordination/CloseRegionCoordination.java   |   69 -
 .../coordination/OpenRegionCoordination.java    |  129 --
 .../coordination/RegionMergeCoordination.java   |  106 --
 .../SplitTransactionCoordination.java           |  101 -
 .../ZKSplitTransactionCoordination.java         |  314 ----
 .../coordination/ZkCloseRegionCoordination.java |  197 --
 .../coordination/ZkCoordinatedStateManager.java |   32 -
 .../coordination/ZkOpenRegionCoordination.java  |  414 -----
 .../coordination/ZkRegionMergeCoordination.java |  326 ----
 .../hadoop/hbase/master/AssignCallable.java     |    2 +-
 .../hadoop/hbase/master/AssignmentManager.java  | 1759 ++----------------
 .../org/apache/hadoop/hbase/master/HMaster.java |   45 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |    4 +-
 .../hadoop/hbase/master/OfflineCallback.java    |  114 --
 .../hadoop/hbase/master/RegionStateStore.java   |   27 +-
 .../hadoop/hbase/master/RegionStates.java       |   39 +-
 .../hadoop/hbase/master/ServerManager.java      |   30 +-
 .../master/handler/ClosedRegionHandler.java     |  108 --
 .../master/handler/OpenedRegionHandler.java     |  103 -
 .../master/handler/ServerShutdownHandler.java   |   31 +-
 .../hadoop/hbase/migration/UpgradeTo96.java     |  259 ---
 .../hbase/regionserver/HRegionServer.java       |   56 +-
 .../hbase/regionserver/RSRpcServices.java       |   23 +-
 .../regionserver/RegionMergeTransaction.java    |  133 +-
 .../hbase/regionserver/SplitTransaction.java    |  125 +-
 .../regionserver/handler/CloseMetaHandler.java  |    9 +-
 .../handler/CloseRegionHandler.java             |   56 +-
 .../regionserver/handler/OpenMetaHandler.java   |    7 +-
 .../regionserver/handler/OpenRegionHandler.java |  105 +-
 .../apache/hadoop/hbase/util/ConfigUtil.java    |   33 -
 .../hadoop/hbase/util/HBaseFsckRepair.java      |    3 +-
 .../src/test/data/TestNamespaceUpgrade.tgz      |  Bin 10449 -> 0 bytes
 .../hadoop/hbase/HBaseTestingUtility.java       |   38 +-
 .../apache/hadoop/hbase/TestDrainingServer.java |  305 ---
 .../org/apache/hadoop/hbase/TestIOFencing.java  |    2 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |    3 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |   10 +-
 .../hadoop/hbase/client/TestReplicasClient.java |   43 +-
 .../client/TestScannersFromClientSide.java      |   37 +-
 .../hbase/io/encoding/TestChangingEncoding.java |    3 +-
 .../org/apache/hadoop/hbase/master/Mocking.java |  110 --
 .../hbase/master/TestAssignmentManager.java     | 1475 ---------------
 .../master/TestAssignmentManagerOnCluster.java  |   43 +-
 .../master/TestDistributedLogSplitting.java     |    8 +-
 .../apache/hadoop/hbase/master/TestMaster.java  |   31 +-
 .../hadoop/hbase/master/TestMasterFailover.java | 1014 +---------
 .../hbase/master/TestMasterNoCluster.java       |   10 +-
 .../TestMasterRestartAfterDisablingTable.java   |   15 +-
 .../hbase/master/TestOpenedRegionHandler.java   |  227 ---
 .../hadoop/hbase/master/TestRestartCluster.java |   47 +-
 .../hadoop/hbase/master/TestRollingRestart.java |   25 +-
 .../master/TestZKBasedOpenCloseRegion.java      |  302 ---
 .../hbase/master/TestZKLessAMOnCluster.java     |   42 -
 .../hbase/migration/TestNamespaceUpgrade.java   |  354 ----
 .../hadoop/hbase/migration/TestUpgradeTo96.java |  270 ---
 .../TestEndToEndSplitTransaction.java           |   26 +-
 .../TestRegionMergeTransactionOnCluster.java    |   16 +-
 .../hbase/regionserver/TestRegionReplicas.java  |   23 +-
 .../regionserver/TestRegionServerNoMaster.java  |  205 +-
 .../TestSplitTransactionOnCluster.java          |  408 +---
 .../regionserver/TestZKLessMergeOnCluster.java  |   45 -
 .../regionserver/TestZKLessSplitOnCluster.java  |   45 -
 .../handler/TestCloseRegionHandler.java         |  255 ---
 .../handler/TestOpenRegionHandler.java          |  360 ----
 .../util/hbck/TestOfflineMetaRebuildBase.java   |    9 +-
 .../util/hbck/TestOfflineMetaRebuildHole.java   |    7 +-
 .../hbck/TestOfflineMetaRebuildOverlap.java     |    7 +-
 78 files changed, 518 insertions(+), 12599 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java
deleted file mode 100644
index 7e7f5c1..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.hbase.util.ByteStringer;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Current state of a region in transition.  Holds state of a region as it moves through the
- * steps that take it from offline to open, etc.  Used by regionserver, master, and zk packages.
- * Encapsulates protobuf serialization/deserialization so we don't leak generated pb outside this
- * class.  Create an instance using createRegionTransition(EventType, byte[], ServerName).
- * <p>Immutable
- */
-@InterfaceAudience.Private
-public class RegionTransition {
-  private final ZooKeeperProtos.RegionTransition rt;
-
-  /**
-   * Shutdown constructor
-   */
-  private RegionTransition() {
-    this(null);
-  }
-
-  private RegionTransition(final ZooKeeperProtos.RegionTransition rt) {
-    this.rt = rt;
-  }
-
-  public EventType getEventType() {
-    return EventType.get(this.rt.getEventTypeCode());
-  }
-
-  public ServerName getServerName() {
-    return ProtobufUtil.toServerName(this.rt.getServerName());
-  }
-
-  public long getCreateTime() {
-    return this.rt.getCreateTime();
-  }
-
-  /**
-   * @return Full region name
-   */
-  public byte [] getRegionName() {
-    return this.rt.getRegionName().toByteArray();
-  }
-
-  public byte [] getPayload() {
-    return this.rt.getPayload().toByteArray();
-  }
-
-  @Override
-  public String toString() {
-    byte [] payload = getPayload();
-    return "region=" + Bytes.toStringBinary(getRegionName()) + ", state=" + getEventType() +
-      ", servername=" + getServerName() + ", createTime=" + this.getCreateTime() +
-      ", payload.length=" + (payload == null? 0: payload.length);
-  }
-
-  /**
-   * @param type
-   * @param regionName
-   * @param sn
-   * @return a serialized pb {@link RegionTransition}
-   */
-  public static RegionTransition createRegionTransition(final EventType type,
-      final byte [] regionName, final ServerName sn) {
-    return createRegionTransition(type, regionName, sn, null);
-  }
-
-  /**
-   * @param type
-   * @param regionName
-   * @param sn
-   * @param payload May be null
-   * @return a serialized pb {@link RegionTransition}
-   */
-  public static RegionTransition createRegionTransition(final EventType type,
-      final byte [] regionName, final ServerName sn, final byte [] payload) {
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName pbsn =
-      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
-        setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
-    ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
-      setEventTypeCode(type.getCode()).setRegionName(ByteStringer.wrap(regionName)).
-        setServerName(pbsn);
-    builder.setCreateTime(System.currentTimeMillis());
-    if (payload != null) builder.setPayload(ByteStringer.wrap(payload));
-    return new RegionTransition(builder.build());
-  }
-
-  /**
-   * @param data Serialized date to parse.
-   * @return A RegionTransition instance made of the passed <code>data</code>
-   * @throws DeserializationException 
-   * @see #toByteArray()
-   */
-  public static RegionTransition parseFrom(final byte [] data) throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(data);
-    try {
-      int prefixLen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.RegionTransition rt = ZooKeeperProtos.RegionTransition.newBuilder().
-        mergeFrom(data, prefixLen, data.length - prefixLen).build();
-      return new RegionTransition(rt);
-    } catch (InvalidProtocolBufferException e) {
-      throw new DeserializationException(e);
-    }
-  }
-
-  /**
-   * @return This instance serialized into a byte array
-   * @see #parseFrom(byte[])
-   */
-  public byte [] toByteArray() {
-    return ProtobufUtil.prependPBMagic(this.rt.toByteArray());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 10e4d04..fe8fb31 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1427,7 +1427,7 @@ public class HBaseAdmin implements Admin {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     CloseRegionRequest request =
-      RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
+      RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
     try {
       CloseRegionResponse response = admin.closeRegion(null, request);
       boolean isRegionClosed = response.getClosed();
@@ -1452,7 +1452,7 @@ public class HBaseAdmin implements Admin {
   throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
-    ProtobufUtil.closeRegion(admin, sn, hri.getRegionName(), false);
+    ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 3289ac1..3830f93 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master;
 
 import java.util.Date;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -28,8 +27,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 
 /**
  * State of a Region while undergoing transitions.
- * Region state cannot be modified except the stamp field.
- * So it is almost immutable.
+ * This class is immutable.
  */
 @InterfaceAudience.Private
 public class RegionState {
@@ -58,16 +56,10 @@ public class RegionState {
                     // master doesn't know it's already created
   }
 
-  // Many threads can update the state at the stamp at the same time
-  private final AtomicLong stamp;
-  private HRegionInfo hri;
-
-  private volatile ServerName serverName;
-  private volatile State state;
-
-  public RegionState() {
-    this.stamp = new AtomicLong(System.currentTimeMillis());
-  }
+  private final long stamp;
+  private final HRegionInfo hri;
+  private final ServerName serverName;
+  private final State state;
 
   public RegionState(HRegionInfo region, State state) {
     this(region, state, System.currentTimeMillis(), null);
@@ -82,20 +74,16 @@ public class RegionState {
       State state, long stamp, ServerName serverName) {
     this.hri = region;
     this.state = state;
-    this.stamp = new AtomicLong(stamp);
+    this.stamp = stamp;
     this.serverName = serverName;
   }
 
-  public void updateTimestampToNow() {
-    setTimestamp(System.currentTimeMillis());
-  }
-
   public State getState() {
     return state;
   }
 
   public long getStamp() {
-    return stamp.get();
+    return stamp;
   }
 
   public HRegionInfo getRegion() {
@@ -248,12 +236,10 @@ public class RegionState {
    * A slower (but more easy-to-read) stringification
    */
   public String toDescriptiveString() {
-    long lstamp = stamp.get();
-    long relTime = System.currentTimeMillis() - lstamp;
-    
+    long relTime = System.currentTimeMillis() - stamp;
     return hri.getRegionNameAsString()
       + " state=" + state
-      + ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)"
+      + ", ts=" + new Date(stamp) + " (" + (relTime/1000) + "s ago)"
       + ", server=" + serverName;
   }
 
@@ -380,10 +366,6 @@ public class RegionState {
     return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null);
   }
 
-  protected void setTimestamp(final long timestamp) {
-    stamp.set(timestamp);
-  }
-
   /**
    * Check if two states are the same, except timestamp
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index eea3b72..4033fb5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -1142,6 +1142,7 @@ public final class ProtobufUtil {
     return toMutation(type, mutation, builder, HConstants.NO_NONCE);
   }
 
+  @SuppressWarnings("deprecation")
   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
       MutationProto.Builder builder, long nonce)
   throws IOException {
@@ -1616,13 +1617,12 @@ public final class ProtobufUtil {
    *
    * @param admin
    * @param regionName
-   * @param transitionInZK
    * @throws IOException
    */
   public static void closeRegion(final AdminService.BlockingInterface admin,
-      final ServerName server, final byte[] regionName, final boolean transitionInZK) throws IOException {
+      final ServerName server, final byte[] regionName) throws IOException {
     CloseRegionRequest closeRegionRequest =
-      RequestConverter.buildCloseRegionRequest(server, regionName, transitionInZK);
+      RequestConverter.buildCloseRegionRequest(server, regionName);
     try {
       admin.closeRegion(null, closeRegionRequest);
     } catch (ServiceException se) {
@@ -1636,18 +1636,15 @@ public final class ProtobufUtil {
    *
    * @param admin
    * @param regionName
-   * @param versionOfClosingNode
    * @return true if the region is closed
    * @throws IOException
    */
   public static boolean closeRegion(final AdminService.BlockingInterface admin,
-      final ServerName server,
-      final byte[] regionName,
-      final int versionOfClosingNode, final ServerName destinationServer,
-      final boolean transitionInZK) throws IOException {
+      final ServerName server, final byte[] regionName,
+      final ServerName destinationServer) throws IOException {
     CloseRegionRequest closeRegionRequest =
       RequestConverter.buildCloseRegionRequest(server,
-        regionName, versionOfClosingNode, destinationServer, transitionInZK);
+        regionName, destinationServer);
     try {
       CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
       return ResponseConverter.isClosed(response);
@@ -1666,7 +1663,7 @@ public final class ProtobufUtil {
   public static void openRegion(final AdminService.BlockingInterface admin,
       ServerName server, final HRegionInfo region) throws IOException {
     OpenRegionRequest request =
-      RequestConverter.buildOpenRegionRequest(server, region, -1, null, null);
+      RequestConverter.buildOpenRegionRequest(server, region, null, null);
     try {
       admin.openRegion(null, request);
     } catch (ServiceException se) {
@@ -2488,6 +2485,7 @@ public final class ProtobufUtil {
     }
   }
 
+  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index e7d0934..d6bcb29 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -22,7 +22,6 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 
-import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -104,7 +103,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionReq
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Triple;
 
 import com.google.protobuf.ByteString;
 
@@ -708,14 +706,12 @@ public final class RequestConverter {
   * @return a protocol buffer OpenRegionRequest
   */
  public static OpenRegionRequest
-     buildOpenRegionRequest(final List<Triple<HRegionInfo, Integer,
+     buildOpenRegionRequest(final List<Pair<HRegionInfo,
          List<ServerName>>> regionOpenInfos, Boolean openForReplay) {
    OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
-   for (Triple<HRegionInfo, Integer, List<ServerName>> regionOpenInfo: regionOpenInfos) {
-     Integer second = regionOpenInfo.getSecond();
-     int versionOfOfflineNode = second == null ? -1 : second.intValue();
-     builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(), versionOfOfflineNode, 
-       regionOpenInfo.getThird(), openForReplay));
+   for (Pair<HRegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
+     builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
+       regionOpenInfo.getSecond(), openForReplay));
    }
    return builder.build();
  }
@@ -725,16 +721,15 @@ public final class RequestConverter {
   *
   * @param server the serverName for the RPC
   * @param region the region to open
-  * @param versionOfOfflineNode that needs to be present in the offline node
   * @param favoredNodes
   * @param openForReplay
   * @return a protocol buffer OpenRegionRequest
   */
  public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
-     final HRegionInfo region, final int versionOfOfflineNode, List<ServerName> favoredNodes,
+     final HRegionInfo region, List<ServerName> favoredNodes,
      Boolean openForReplay) {
    OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
-   builder.addOpenInfo(buildRegionOpenInfo(region, versionOfOfflineNode, favoredNodes, 
+   builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes,
      openForReplay));
    if (server != null) {
      builder.setServerStartCode(server.getStartcode());
@@ -765,33 +760,21 @@ public final class RequestConverter {
   * Create a CloseRegionRequest for a given region name
   *
   * @param regionName the name of the region to close
-  * @param transitionInZK indicator if to transition in ZK
   * @return a CloseRegionRequest
   */
  public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
-     final byte[] regionName, final boolean transitionInZK) {
-   CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
-   RegionSpecifier region = buildRegionSpecifier(
-     RegionSpecifierType.REGION_NAME, regionName);
-   builder.setRegion(region);
-   builder.setTransitionInZK(transitionInZK);
-   if (server != null) {
-     builder.setServerStartCode(server.getStartcode());
-   }
-   return builder.build();
+     final byte[] regionName) {
+   return buildCloseRegionRequest(server, regionName, null);
  }
 
   public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
-    final byte[] regionName, final int versionOfClosingNode,
-    ServerName destinationServer, final boolean transitionInZK) {
+    final byte[] regionName, ServerName destinationServer) {
     CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
     RegionSpecifier region = buildRegionSpecifier(
       RegionSpecifierType.REGION_NAME, regionName);
     builder.setRegion(region);
-    builder.setVersionOfClosingNode(versionOfClosingNode);
-    builder.setTransitionInZK(transitionInZK);
     if (destinationServer != null){
-      builder.setDestinationServer(ProtobufUtil.toServerName( destinationServer) );
+      builder.setDestinationServer(ProtobufUtil.toServerName(destinationServer));
     }
     if (server != null) {
       builder.setServerStartCode(server.getStartcode());
@@ -803,18 +786,15 @@ public final class RequestConverter {
   * Create a CloseRegionRequest for a given encoded region name
   *
   * @param encodedRegionName the name of the region to close
-  * @param transitionInZK indicator if to transition in ZK
   * @return a CloseRegionRequest
   */
  public static CloseRegionRequest
-     buildCloseRegionRequest(ServerName server, final String encodedRegionName,
-       final boolean transitionInZK) {
+     buildCloseRegionRequest(ServerName server, final String encodedRegionName) {
    CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
    RegionSpecifier region = buildRegionSpecifier(
      RegionSpecifierType.ENCODED_REGION_NAME,
      Bytes.toBytes(encodedRegionName));
    builder.setRegion(region);
-   builder.setTransitionInZK(transitionInZK);
    if (server != null) {
      builder.setServerStartCode(server.getStartcode());
    }
@@ -1498,13 +1478,10 @@ public final class RequestConverter {
    * Create a RegionOpenInfo based on given region info and version of offline node
    */
   private static RegionOpenInfo buildRegionOpenInfo(
-      final HRegionInfo region, final int versionOfOfflineNode,
+      final HRegionInfo region,
       final List<ServerName> favoredNodes, Boolean openForReplay) {
     RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
     builder.setRegion(HRegionInfo.convert(region));
-    if (versionOfOfflineNode >= 0) {
-      builder.setVersionOfOfflineNode(versionOfOfflineNode);
-    }
     if (favoredNodes != null) {
       for (ServerName server : favoredNodes) {
         builder.addFavoredNodes(ProtobufUtil.toServerName(server));

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
deleted file mode 100644
index d85496c..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
+++ /dev/null
@@ -1,1057 +0,0 @@
-/**
- *
- * 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.zookeeper;
-
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.Code;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.data.Stat;
-
-// We should not be importing this Type here, nor a RegionTransition, etc.  This class should be
-// about zk and bytes only.
-
-/**
- * Utility class for doing region assignment in ZooKeeper.  This class extends
- * stuff done in {@link ZKUtil} to cover specific assignment operations.
- * <p>
- * Contains only static methods and constants.
- * <p>
- * Used by both the Master and RegionServer.
- * <p>
- * All valid transitions outlined below:
- * <p>
- * <b>MASTER</b>
- * <ol>
- *   <li>
- *     Master creates an unassigned node as OFFLINE.
- *     - Cluster startup and table enabling.
- *   </li>
- *   <li>
- *     Master forces an existing unassigned node to OFFLINE.
- *     - RegionServer failure.
- *     - Allows transitions from all states to OFFLINE.
- *   </li>
- *   <li>
- *     Master deletes an unassigned node that was in a OPENED state.
- *     - Normal region transitions.  Besides cluster startup, no other deletions
- *     of unassigned nodes is allowed.
- *   </li>
- *   <li>
- *     Master deletes all unassigned nodes regardless of state.
- *     - Cluster startup before any assignment happens.
- *   </li>
- * </ol>
- * <p>
- * <b>REGIONSERVER</b>
- * <ol>
- *   <li>
- *     RegionServer creates an unassigned node as CLOSING.
- *     - All region closes will do this in response to a CLOSE RPC from Master.
- *     - A node can never be transitioned to CLOSING, only created.
- *   </li>
- *   <li>
- *     RegionServer transitions an unassigned node from CLOSING to CLOSED.
- *     - Normal region closes.  CAS operation.
- *   </li>
- *   <li>
- *     RegionServer transitions an unassigned node from OFFLINE to OPENING.
- *     - All region opens will do this in response to an OPEN RPC from the Master.
- *     - Normal region opens.  CAS operation.
- *   </li>
- *   <li>
- *     RegionServer transitions an unassigned node from OPENING to OPENED.
- *     - Normal region opens.  CAS operation.
- *   </li>
- * </ol>
- */
-@InterfaceAudience.Private
-public class ZKAssign {
-  private static final Log LOG = LogFactory.getLog(ZKAssign.class);
-
-  /**
-   * Gets the full path node name for the unassigned node for the specified
-   * region.
-   * @param zkw zk reference
-   * @param regionName region name
-   * @return full path node name
-   */
-  public static String getNodeName(ZooKeeperWatcher zkw, String regionName) {
-    return ZKUtil.joinZNode(zkw.assignmentZNode, regionName);
-  }
-
-  /**
-   * Gets the region name from the full path node name of an unassigned node.
-   * @param path full zk path
-   * @return region name
-   */
-  public static String getRegionName(ZooKeeperWatcher zkw, String path) {
-    return path.substring(zkw.assignmentZNode.length()+1);
-  }
-
-  // Master methods
-
-  /**
-   * Creates a new unassigned node in the OFFLINE state for the specified region.
-   *
-   * <p>Does not transition nodes from other states.  If a node already exists
-   * for this region, a {@link NodeExistsException} will be thrown.
-   *
-   * <p>Sets a watcher on the unassigned region node if the method is successful.
-   *
-   * <p>This method should only be used during cluster startup and the enabling
-   * of a table.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as offline
-   * @param serverName server transition will happen on
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NodeExistsException if node already exists
-   */
-  public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
-      ServerName serverName)
-  throws KeeperException, KeeperException.NodeExistsException {
-    createNodeOffline(zkw, region, serverName, EventType.M_ZK_REGION_OFFLINE);
-  }
-
-  public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
-      ServerName serverName, final EventType event)
-  throws KeeperException, KeeperException.NodeExistsException {
-    LOG.debug(zkw.prefix("Creating unassigned node " +
-      region.getEncodedName() + " in OFFLINE state"));
-    RegionTransition rt =
-      RegionTransition.createRegionTransition(event, region.getRegionName(), serverName);
-    String node = getNodeName(zkw, region.getEncodedName());
-    ZKUtil.createAndWatch(zkw, node, rt.toByteArray());
-  }
-
-  /**
-   * Creates an unassigned node in the OFFLINE state for the specified region.
-   * <p>
-   * Runs asynchronously.  Depends on no pre-existing znode.
-   *
-   * <p>Sets a watcher on the unassigned region node.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as offline
-   * @param serverName server transition will happen on
-   * @param cb
-   * @param ctx
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NodeExistsException if node already exists
-   */
-  public static void asyncCreateNodeOffline(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName,
-      final AsyncCallback.StringCallback cb, final Object ctx)
-  throws KeeperException {
-    LOG.debug(zkw.prefix("Async create of unassigned node " +
-      region.getEncodedName() + " with OFFLINE state"));
-    RegionTransition rt =
-      RegionTransition.createRegionTransition(
-          EventType.M_ZK_REGION_OFFLINE, region.getRegionName(), serverName);
-    String node = getNodeName(zkw, region.getEncodedName());
-    ZKUtil.asyncCreate(zkw, node, rt.toByteArray(), cb, ctx);
-  }
-
-  /**
-   * Creates or force updates an unassigned node to the OFFLINE state for the
-   * specified region.
-   * <p>
-   * Attempts to create the node but if it exists will force it to transition to
-   * and OFFLINE state.
-   *
-   * <p>Sets a watcher on the unassigned region node if the method is
-   * successful.
-   *
-   * <p>This method should be used when assigning a region.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as offline
-   * @param serverName server transition will happen on
-   * @return the version of the znode created in OFFLINE state, -1 if
-   *         unsuccessful.
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NodeExistsException if node already exists
-   */
-  public static int createOrForceNodeOffline(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName) throws KeeperException {
-    LOG.debug(zkw.prefix("Creating (or updating) unassigned node " +
-      region.getEncodedName() + " with OFFLINE state"));
-    RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_OFFLINE,
-      region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
-    byte [] data = rt.toByteArray();
-    String node = getNodeName(zkw, region.getEncodedName());
-    zkw.sync(node);
-    int version = ZKUtil.checkExists(zkw, node);
-    if (version == -1) {
-      return ZKUtil.createAndWatch(zkw, node, data);
-    } else {
-      boolean setData = false;
-      try {
-        setData = ZKUtil.setData(zkw, node, data, version);
-        // Setdata throws KeeperException which aborts the Master. So we are
-        // catching it here.
-        // If just before setting the znode to OFFLINE if the RS has made any
-        // change to the
-        // znode state then we need to return -1.
-      } catch (KeeperException kpe) {
-        LOG.info("Version mismatch while setting the node to OFFLINE state.");
-        return -1;
-      }
-      if (!setData) {
-        return -1;
-      } else {
-        // We successfully forced to OFFLINE, reset watch and handle if
-        // the state changed in between our set and the watch
-        byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-        rt = getRegionTransition(bytes);
-        if (rt.getEventType() != EventType.M_ZK_REGION_OFFLINE) {
-          // state changed, need to process
-          return -1;
-        }
-      }
-    }
-    return version + 1;
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the OPENED state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used during normal region transitions when a region
-   * finishes successfully opening.  This is the Master acknowledging completion
-   * of the specified regions transition.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName opened region to be deleted from zk
-   * @param sn the expected region transition target server name
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteOpenedNode(ZooKeeperWatcher zkw,
-      String encodedRegionName, ServerName sn)
-  throws KeeperException, KeeperException.NoNodeException {
-    return deleteNode(zkw, encodedRegionName,
-      EventType.RS_ZK_REGION_OPENED, sn);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the OFFLINE state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used during master failover when the regions on an RS
-   * that has died are all set to OFFLINE before being processed.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName closed region to be deleted from zk
-   * @param sn the expected region transition target server name
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteOfflineNode(ZooKeeperWatcher zkw,
-      String encodedRegionName, ServerName sn)
-  throws KeeperException, KeeperException.NoNodeException {
-    return deleteNode(zkw, encodedRegionName,
-      EventType.M_ZK_REGION_OFFLINE, sn);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the CLOSED state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used during table disables when a region finishes
-   * successfully closing.  This is the Master acknowledging completion
-   * of the specified regions transition to being closed.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName closed region to be deleted from zk
-   * @param sn the expected region transition target server name
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteClosedNode(ZooKeeperWatcher zkw,
-      String encodedRegionName, ServerName sn)
-  throws KeeperException, KeeperException.NoNodeException {
-    return deleteNode(zkw, encodedRegionName,
-      EventType.RS_ZK_REGION_CLOSED, sn);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the CLOSING state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used during table disables when a region finishes
-   * successfully closing.  This is the Master acknowledging completion
-   * of the specified regions transition to being closed.
-   *
-   * @param zkw zk reference
-   * @param region closing region to be deleted from zk
-   * @param sn the expected region transition target server name
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteClosingNode(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName sn)
-  throws KeeperException, KeeperException.NoNodeException {
-    String encodedRegionName = region.getEncodedName();
-    return deleteNode(zkw, encodedRegionName,
-      EventType.M_ZK_REGION_CLOSING, sn);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the specified state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used when a region finishes opening/closing.
-   * The Master acknowledges completion
-   * of the specified regions transition to being closed/opened.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName region to be deleted from zk
-   * @param expectedState state region must be in for delete to complete
-   * @param sn the expected region transition target server name
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName,
-      EventType expectedState, ServerName sn)
-  throws KeeperException, KeeperException.NoNodeException {
-    return deleteNode(zkw, encodedRegionName, expectedState, sn, -1);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the specified state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used when a region finishes opening/closing.
-   * The Master acknowledges completion
-   * of the specified regions transition to being closed/opened.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName region to be deleted from zk
-   * @param expectedState state region must be in for delete to complete
-   * @param expectedVersion of the znode that is to be deleted.
-   *        If expectedVersion need not be compared while deleting the znode
-   *        pass -1
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName,
-      EventType expectedState, int expectedVersion)
-  throws KeeperException, KeeperException.NoNodeException {
-    return deleteNode(zkw, encodedRegionName, expectedState, null, expectedVersion);
-  }
-
-  /**
-   * Deletes an existing unassigned node that is in the specified state for the
-   * specified region.
-   *
-   * <p>If a node does not already exist for this region, a
-   * {@link NoNodeException} will be thrown.
-   *
-   * <p>No watcher is set whether this succeeds or not.
-   *
-   * <p>Returns false if the node was not in the proper state but did exist.
-   *
-   * <p>This method is used when a region finishes opening/closing.
-   * The Master acknowledges completion
-   * of the specified regions transition to being closed/opened.
-   *
-   * @param zkw zk reference
-   * @param encodedRegionName region to be deleted from zk
-   * @param expectedState state region must be in for delete to complete
-   * @param serverName the expected region transition target server name
-   * @param expectedVersion of the znode that is to be deleted.
-   *        If expectedVersion need not be compared while deleting the znode
-   *        pass -1
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NoNodeException if node does not exist
-   */
-  public static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName,
-      EventType expectedState, ServerName serverName, int expectedVersion)
-  throws KeeperException, KeeperException.NoNodeException {
-    if (LOG.isTraceEnabled()) {
-    	LOG.trace(zkw.prefix("Deleting existing unassigned " +
-      "node " + encodedRegionName + " in expected state " + expectedState));
-    }
-    String node = getNodeName(zkw, encodedRegionName);
-    zkw.sync(node);
-    Stat stat = new Stat();
-    byte [] bytes = ZKUtil.getDataNoWatch(zkw, node, stat);
-    if (bytes == null) {
-      // If it came back null, node does not exist.
-      throw KeeperException.create(Code.NONODE);
-    }
-    RegionTransition rt = getRegionTransition(bytes);
-    EventType et = rt.getEventType();
-    if (!et.equals(expectedState)) {
-      LOG.warn(zkw.prefix("Attempting to delete unassigned node " + encodedRegionName + " in " +
-        expectedState + " state but node is in " + et + " state"));
-      return false;
-    }
-    // Verify the server transition happens on is not changed
-    if (serverName != null && !rt.getServerName().equals(serverName)) {
-      LOG.warn(zkw.prefix("Attempting to delete unassigned node " + encodedRegionName
-        + " with target " + serverName + " but node has " + rt.getServerName()));
-      return false;
-    }
-    if (expectedVersion != -1
-        && stat.getVersion() != expectedVersion) {
-      LOG.warn("The node " + encodedRegionName + " we are trying to delete is not" +
-        " the expected one. Got a version mismatch");
-      return false;
-    }
-    if(!ZKUtil.deleteNode(zkw, node, stat.getVersion())) {
-      LOG.warn(zkw.prefix("Attempting to delete " +
-          "unassigned node " + encodedRegionName + " in " + expectedState +
-          " state but after verifying state, we got a version mismatch"));
-      return false;
-    }
-    LOG.debug(zkw.prefix("Deleted unassigned node " +
-        encodedRegionName + " in expected state " + expectedState));
-    return true;
-  }
-
-  /**
-   * Deletes all unassigned nodes regardless of their state.
-   *
-   * <p>No watchers are set.
-   *
-   * <p>This method is used by the Master during cluster startup to clear out
-   * any existing state from other cluster runs.
-   *
-   * @param zkw zk reference
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void deleteAllNodes(ZooKeeperWatcher zkw)
-  throws KeeperException {
-    LOG.debug(zkw.prefix("Deleting any existing unassigned nodes"));
-    ZKUtil.deleteChildrenRecursively(zkw, zkw.assignmentZNode);
-  }
-
-  /**
-   * Creates a new unassigned node in the CLOSING state for the specified
-   * region.
-   *
-   * <p>Does not transition nodes from any states.  If a node already exists
-   * for this region, a {@link NodeExistsException} will be thrown.
-   *
-   * <p>If creation is successful, returns the version number of the CLOSING
-   * node created.
-   *
-   * <p>Set a watch.
-   *
-   * <p>This method should only be used by a Master when initiating a
-   * close of a region before sending a close request to the region server.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as closing
-   * @param serverName server transition will happen on
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NodeExistsException if node already exists
-   */
-  public static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region,
-      ServerName serverName)
-  throws KeeperException, KeeperException.NodeExistsException {
-    LOG.debug(zkw.prefix("Creating unassigned node " +
-      region.getEncodedName() + " in a CLOSING state"));
-    RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
-      region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
-    String node = getNodeName(zkw, region.getEncodedName());
-    return ZKUtil.createAndWatch(zkw, node, rt.toByteArray());
-  }
-
-  // RegionServer methods
-
-  /**
-   * Transitions an existing unassigned node for the specified region which is
-   * currently in the CLOSING state to be in the CLOSED state.
-   *
-   * <p>Does not transition nodes from other states.  If for some reason the
-   * node could not be transitioned, the method returns -1.  If the transition
-   * is successful, the version of the node after transition is returned.
-   *
-   * <p>This method can fail and return false for three different reasons:
-   * <ul><li>Unassigned node for this region does not exist</li>
-   * <li>Unassigned node for this region is not in CLOSING state</li>
-   * <li>After verifying CLOSING state, update fails because of wrong version
-   * (someone else already transitioned the node)</li>
-   * </ul>
-   *
-   * <p>Does not set any watches.
-   *
-   * <p>This method should only be used by a RegionServer when initiating a
-   * close of a region after receiving a CLOSE RPC from the Master.
-   *
-   * @param zkw zk reference
-   * @param region region to be transitioned to closed
-   * @param serverName server transition happens on
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int transitionNodeClosed(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName, int expectedVersion)
-  throws KeeperException {
-    return transitionNode(zkw, region, serverName,
-        EventType.M_ZK_REGION_CLOSING,
-        EventType.RS_ZK_REGION_CLOSED, expectedVersion);
-  }
-
-  /**
-   * Transitions an existing unassigned node for the specified region which is
-   * currently in the OFFLINE state to be in the OPENING state.
-   *
-   * <p>Does not transition nodes from other states.  If for some reason the
-   * node could not be transitioned, the method returns -1.  If the transition
-   * is successful, the version of the node written as OPENING is returned.
-   *
-   * <p>This method can fail and return -1 for three different reasons:
-   * <ul><li>Unassigned node for this region does not exist</li>
-   * <li>Unassigned node for this region is not in OFFLINE state</li>
-   * <li>After verifying OFFLINE state, update fails because of wrong version
-   * (someone else already transitioned the node)</li>
-   * </ul>
-   *
-   * <p>Does not set any watches.
-   *
-   * <p>This method should only be used by a RegionServer when initiating an
-   * open of a region after receiving an OPEN RPC from the Master.
-   *
-   * @param zkw zk reference
-   * @param region region to be transitioned to opening
-   * @param serverName server transition happens on
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int transitionNodeOpening(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName)
-  throws KeeperException {
-    return transitionNodeOpening(zkw, region, serverName,
-      EventType.M_ZK_REGION_OFFLINE);
-  }
-
-  public static int transitionNodeOpening(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName, final EventType beginState)
-  throws KeeperException {
-    return transitionNode(zkw, region, serverName, beginState,
-      EventType.RS_ZK_REGION_OPENING, -1);
-  }
-
-  /**
-   * Confirm an existing unassigned node for the specified region which is
-   * currently in the OPENING state to be still in the OPENING state on
-   * the specified server.
-   *
-   * <p>If for some reason the check fails, the method returns -1. Otherwise,
-   * the version of the node (same as the expected version) is returned.
-   *
-   * <p>This method can fail and return -1 for three different reasons:
-   * <ul><li>Unassigned node for this region does not exist</li>
-   * <li>Unassigned node for this region is not in OPENING state</li>
-   * <li>After verifying OPENING state, the server name or the version of the
-   * doesn't match)</li>
-   * </ul>
-   *
-   * <p>Does not set any watches.
-   *
-   * <p>This method should only be used by a RegionServer when initiating an
-   * open of a region after receiving an OPEN RPC from the Master.
-   *
-   * @param zkw zk reference
-   * @param region region to be transitioned to opening
-   * @param serverName server transition happens on
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int confirmNodeOpening(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName, int expectedVersion)
-  throws KeeperException {
-
-    String encoded = region.getEncodedName();
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(zkw.prefix("Attempting to retransition opening state of node " +
-          HRegionInfo.prettyPrint(encoded)));
-    }
-
-    String node = getNodeName(zkw, encoded);
-    zkw.sync(node);
-
-    // Read existing data of the node
-    Stat stat = new Stat();
-    byte [] existingBytes = ZKUtil.getDataNoWatch(zkw, node, stat);
-    if (existingBytes == null) {
-      // Node no longer exists.  Return -1. It means unsuccessful transition.
-      return -1;
-    }
-    RegionTransition rt = getRegionTransition(existingBytes);
-
-    // Verify it is the expected version
-    if (expectedVersion != -1 && stat.getVersion() != expectedVersion) {
-      LOG.warn(zkw.prefix("Attempt to retransition the opening state of the " +
-          "unassigned node for " + encoded + " failed, " +
-          "the node existed but was version " + stat.getVersion() +
-          " not the expected version " + expectedVersion));
-      return -1;
-    }
-
-    // Verify it is in expected state
-    EventType et = rt.getEventType();
-    if (!et.equals(EventType.RS_ZK_REGION_OPENING)) {
-      String existingServer = (rt.getServerName() == null)
-          ? "<unknown>" : rt.getServerName().toString();
-      LOG.warn(zkw.prefix("Attempt to retransition the opening state of the unassigned node for "
-          + encoded + " failed, the node existed but was in the state " + et +
-          " set by the server " + existingServer));
-      return -1;
-    }
-
-    return expectedVersion;
-  }
-
-  /**
-   * Transitions an existing unassigned node for the specified region which is
-   * currently in the OPENING state to be in the OPENED state.
-   *
-   * <p>Does not transition nodes from other states.  If for some reason the
-   * node could not be transitioned, the method returns -1.  If the transition
-   * is successful, the version of the node after transition is returned.
-   *
-   * <p>This method can fail and return false for three different reasons:
-   * <ul><li>Unassigned node for this region does not exist</li>
-   * <li>Unassigned node for this region is not in OPENING state</li>
-   * <li>After verifying OPENING state, update fails because of wrong version
-   * (this should never actually happen since an RS only does this transition
-   * following a transition to OPENING.  if two RS are conflicting, one would
-   * fail the original transition to OPENING and not this transition)</li>
-   * </ul>
-   *
-   * <p>Does not set any watches.
-   *
-   * <p>This method should only be used by a RegionServer when completing the
-   * open of a region.
-   *
-   * @param zkw zk reference
-   * @param region region to be transitioned to opened
-   * @param serverName server transition happens on
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int transitionNodeOpened(ZooKeeperWatcher zkw,
-      HRegionInfo region, ServerName serverName, int expectedVersion)
-  throws KeeperException {
-    return transitionNode(zkw, region, serverName,
-        EventType.RS_ZK_REGION_OPENING,
-        EventType.RS_ZK_REGION_OPENED, expectedVersion);
-  }
-
-  /**
-   *
-   * @param zkw zk reference
-   * @param region region to be closed
-   * @param expectedVersion expected version of the znode
-   * @return true if the znode exists, has the right version and the right state. False otherwise.
-   * @throws KeeperException
-   */
-  public static boolean checkClosingState(ZooKeeperWatcher zkw, HRegionInfo region,
-                                          int expectedVersion) throws KeeperException {
-
-    final String encoded = getNodeName(zkw, region.getEncodedName());
-    zkw.sync(encoded);
-
-    // Read existing data of the node
-    Stat stat = new Stat();
-    byte[] existingBytes = ZKUtil.getDataNoWatch(zkw, encoded, stat);
-
-    if (existingBytes == null) {
-      LOG.warn(zkw.prefix("Attempt to check the " +
-          "closing node for " + encoded +
-          ". The node does not exist"));
-      return false;
-    }
-
-    if (expectedVersion != -1 && stat.getVersion() != expectedVersion) {
-      LOG.warn(zkw.prefix("Attempt to check the " +
-          "closing node for " + encoded +
-          ". The node existed but was version " + stat.getVersion() +
-          " not the expected version " + expectedVersion));
-      return false;
-    }
-
-    RegionTransition rt = getRegionTransition(existingBytes);
-
-    if (!EventType.M_ZK_REGION_CLOSING.equals(rt.getEventType())) {
-      LOG.warn(zkw.prefix("Attempt to check the " +
-          "closing node for " + encoded +
-          ". The node existed but was in an unexpected state: " + rt.getEventType()));
-      return false;
-    }
-
-    return true;
-  }
-
-  /**
-   * Method that actually performs unassigned node transitions.
-   *
-   * <p>Attempts to transition the unassigned node for the specified region
-   * from the expected state to the state in the specified transition data.
-   *
-   * <p>Method first reads existing data and verifies it is in the expected
-   * state.  If the node does not exist or the node is not in the expected
-   * state, the method returns -1.  If the transition is successful, the
-   * version number of the node following the transition is returned.
-   *
-   * <p>If the read state is what is expected, it attempts to write the new
-   * state and data into the node.  When doing this, it includes the expected
-   * version (determined when the existing state was verified) to ensure that
-   * only one transition is successful.  If there is a version mismatch, the
-   * method returns -1.
-   *
-   * <p>If the write is successful, no watch is set and the method returns true.
-   *
-   * @param zkw zk reference
-   * @param region region to be transitioned to opened
-   * @param serverName server transition happens on
-   * @param endState state to transition node to if all checks pass
-   * @param beginState state the node must currently be in to do transition
-   * @param expectedVersion expected version of data before modification, or -1
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region,
-      ServerName serverName, EventType beginState, EventType endState,
-      int expectedVersion)
-  throws KeeperException {
-    return transitionNode(zkw, region, serverName, beginState, endState, expectedVersion, null);
-  }
-
-
-  public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region,
-      ServerName serverName, EventType beginState, EventType endState,
-      int expectedVersion, final byte [] payload)
-  throws KeeperException {
-    String encoded = region.getEncodedName();
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(zkw.prefix("Transitioning " + HRegionInfo.prettyPrint(encoded) +
-        " from " + beginState.toString() + " to " + endState.toString()));
-    }
-
-    String node = getNodeName(zkw, encoded);
-    zkw.sync(node);
-
-    // Read existing data of the node
-    Stat stat = new Stat();
-    byte [] existingBytes = ZKUtil.getDataNoWatch(zkw, node, stat);
-    if (existingBytes == null) {
-      // Node no longer exists.  Return -1. It means unsuccessful transition.
-      return -1;
-    }
-
-    // Verify it is the expected version
-    if (expectedVersion != -1 && stat.getVersion() != expectedVersion) {
-      LOG.warn(zkw.prefix("Attempt to transition the " +
-        "unassigned node for " + encoded +
-        " from " + beginState + " to " + endState + " failed, " +
-        "the node existed but was version " + stat.getVersion() +
-        " not the expected version " + expectedVersion));
-        return -1;
-    }
-
-    if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
-        && endState.equals(EventType.RS_ZK_REGION_OPENING)
-        && expectedVersion == -1 && stat.getVersion() != 0) {
-      // the below check ensures that double assignment doesnot happen.
-      // When the node is created for the first time then the expected version
-      // that is passed will be -1 and the version in znode will be 0.
-      // In all other cases the version in znode will be > 0.
-      LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
-          + encoded + " from " + beginState + " to " + endState + " failed, "
-          + "the node existed but was version " + stat.getVersion()
-          + " not the expected version " + expectedVersion));
-      return -1;
-    }
-
-    RegionTransition rt = getRegionTransition(existingBytes);
-
-    // Verify the server transition happens on is not changed
-    if (!rt.getServerName().equals(serverName)) {
-      LOG.warn(zkw.prefix("Attempt to transition the " +
-        "unassigned node for " + encoded +
-        " from " + beginState + " to " + endState + " failed, " +
-        "the server that tried to transition was " + serverName +
-        " not the expected " + rt.getServerName()));
-      return -1;
-    }
-
-    // Verify it is in expected state
-    EventType et = rt.getEventType();
-    if (!et.equals(beginState)) {
-      String existingServer = (rt.getServerName() == null)
-        ? "<unknown>" : rt.getServerName().toString();
-      LOG.warn(zkw.prefix("Attempt to transition the unassigned node for " + encoded
-        + " from " + beginState + " to " + endState + " failed, the node existed but"
-        + " was in the state " + et + " set by the server " + existingServer));
-      return -1;
-    }
-
-    // Write new data, ensuring data has not changed since we last read it
-    try {
-      rt = RegionTransition.createRegionTransition(
-          endState, region.getRegionName(), serverName, payload);
-      if(!ZKUtil.setData(zkw, node, rt.toByteArray(), stat.getVersion())) {
-        LOG.warn(zkw.prefix("Attempt to transition the " +
-        "unassigned node for " + encoded +
-        " from " + beginState + " to " + endState + " failed, " +
-        "the node existed and was in the expected state but then when " +
-        "setting data we got a version mismatch"));
-        return -1;
-      }
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(zkw.prefix("Transitioned node " + encoded +
-          " from " + beginState + " to " + endState));
-      }
-      return stat.getVersion() + 1;
-    } catch (KeeperException.NoNodeException nne) {
-      LOG.warn(zkw.prefix("Attempt to transition the " +
-        "unassigned node for " + encoded +
-        " from " + beginState + " to " + endState + " failed, " +
-        "the node existed and was in the expected state but then when " +
-        "setting data it no longer existed"));
-      return -1;
-    }
-  }
-
-  private static RegionTransition getRegionTransition(final byte [] bytes) throws KeeperException {
-    try {
-      return RegionTransition.parseFrom(bytes);
-    } catch (DeserializationException e) {
-      // Convert to a zk exception for now.  Otherwise have to change API
-      throw ZKUtil.convert(e);
-    }
-  }
-
-  /**
-   * Gets the current data in the unassigned node for the specified region name
-   * or fully-qualified path.
-   *
-   * <p>Returns null if the region does not currently have a node.
-   *
-   * <p>Sets a watch on the node if the node exists.
-   *
-   * @param zkw zk reference
-   * @param pathOrRegionName fully-specified path or region name
-   * @return znode content
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte [] getData(ZooKeeperWatcher zkw,
-      String pathOrRegionName)
-  throws KeeperException {
-    String node = getPath(zkw, pathOrRegionName);
-    return ZKUtil.getDataAndWatch(zkw, node);
-  }
-
-  /**
-   * Gets the current data in the unassigned node for the specified region name
-   * or fully-qualified path.
-   *
-   * <p>Returns null if the region does not currently have a node.
-   *
-   * <p>Sets a watch on the node if the node exists.
-   *
-   * @param zkw zk reference
-   * @param pathOrRegionName fully-specified path or region name
-   * @param stat object to populate the version.
-   * @return znode content
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte [] getDataAndWatch(ZooKeeperWatcher zkw,
-      String pathOrRegionName, Stat stat)
-  throws KeeperException {
-    String node = getPath(zkw, pathOrRegionName);
-    return ZKUtil.getDataAndWatch(zkw, node, stat);
-  }
-
-  /**
-   * Gets the current data in the unassigned node for the specified region name
-   * or fully-qualified path.
-   *
-   * <p>Returns null if the region does not currently have a node.
-   *
-   * <p>Does not set a watch.
-   *
-   * @param zkw zk reference
-   * @param pathOrRegionName fully-specified path or region name
-   * @param stat object to store node info into on getData call
-   * @return znode content
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte [] getDataNoWatch(ZooKeeperWatcher zkw,
-      String pathOrRegionName, Stat stat)
-  throws KeeperException {
-    String node = getPath(zkw, pathOrRegionName);
-    return ZKUtil.getDataNoWatch(zkw, node, stat);
-  }
-
-  /**
-   * @param zkw
-   * @param pathOrRegionName
-   * @return Path to znode
-   */
-  public static String getPath(final ZooKeeperWatcher zkw, final String pathOrRegionName) {
-    return pathOrRegionName.startsWith("/")? pathOrRegionName : getNodeName(zkw, pathOrRegionName);
-  }
-
-  /**
-   * Get the version of the specified znode
-   * @param zkw zk reference
-   * @param region region's info
-   * @return the version of the znode, -1 if it doesn't exist
-   * @throws KeeperException
-   */
-  public static int getVersion(ZooKeeperWatcher zkw, HRegionInfo region)
-    throws KeeperException {
-    String znode = getNodeName(zkw, region.getEncodedName());
-    return ZKUtil.checkExists(zkw, znode);
-  }
-
-  /**
-   * Delete the assignment node regardless of its current state.
-   * <p>
-   * Fail silent even if the node does not exist at all.
-   * @param watcher
-   * @param regionInfo
-   * @throws KeeperException
-   */
-  public static void deleteNodeFailSilent(ZooKeeperWatcher watcher,
-      HRegionInfo regionInfo)
-  throws KeeperException {
-    String node = getNodeName(watcher, regionInfo.getEncodedName());
-    ZKUtil.deleteNodeFailSilent(watcher, node);
-  }
-
-  /**
-   * Blocks until there are no node in regions in transition.
-   * <p>
-   * Used in testing only.
-   * @param zkw zk reference
-   * @throws KeeperException
-   * @throws InterruptedException
-   */
-  public static void blockUntilNoRIT(ZooKeeperWatcher zkw)
-  throws KeeperException, InterruptedException {
-    while (ZKUtil.nodeHasChildren(zkw, zkw.assignmentZNode)) {
-      List<String> znodes =
-        ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.assignmentZNode);
-      if (znodes != null && !znodes.isEmpty()) {
-        LOG.debug("Waiting on RIT: " + znodes);
-      }
-      Thread.sleep(100);
-    }
-  }
-
-  /**
-   * Blocks until there is at least one node in regions in transition.
-   * <p>
-   * Used in testing only.
-   * @param zkw zk reference
-   * @throws KeeperException
-   * @throws InterruptedException
-   */
-  public static void blockUntilRIT(ZooKeeperWatcher zkw)
-  throws KeeperException, InterruptedException {
-    while (!ZKUtil.nodeHasChildren(zkw, zkw.assignmentZNode)) {
-      List<String> znodes =
-        ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.assignmentZNode);
-      if (znodes == null || znodes.isEmpty()) {
-        LOG.debug("No RIT in ZK");
-      }
-      Thread.sleep(100);
-    }
-  }
-
-  /**
-   * Presume bytes are serialized unassigned data structure
-   * @param znodeBytes
-   * @return String of the deserialized znode bytes.
-   */
-  static String toString(final byte[] znodeBytes) {
-    // This method should not exist.  Used by ZKUtil stringifying RegionTransition.  Have the
-    // method in here so RegionTransition does not leak into ZKUtil.
-    try {
-      RegionTransition rt = RegionTransition.parseFrom(znodeBytes);
-      return rt.toString();
-    } catch (DeserializationException e) {
-      return "";
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 7836d11..b13667f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -263,10 +263,6 @@ public class ZKUtil {
     private final String keytabFile;
     private final String principal;
 
-    public JaasConfiguration(String loginContextName, String principal) {
-      this(loginContextName, principal, null, true);
-    }
-
     public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
       this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
     }
@@ -951,7 +947,7 @@ public class ZKUtil {
          conf.get("hbase.zookeeper.client.keytab.file") != null);
   }
 
-  private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
+  private static List<ACL> createACL(ZooKeeperWatcher zkw, String node) {
     if (isSecureZooKeeper(zkw.getConfiguration())) {
       // Certain znodes are accessed directly by the client,
       // so they must be readable by non-authenticated clients
@@ -961,7 +957,6 @@ public class ZKUtil {
           (node.equals(zkw.clusterIdZNode) == true) ||
           (node.equals(zkw.rsZNode) == true) ||
           (node.equals(zkw.backupMasterAddressesZNode) == true) ||
-          (node.startsWith(zkw.assignmentZNode) == true) ||
           (node.startsWith(zkw.tableZNode) == true)) {
         return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
       }
@@ -1779,8 +1774,6 @@ public class ZKUtil {
       " byte(s) of data from znode " + znode +
       (watcherSet? " and set watcher; ": "; data=") +
       (data == null? "null": data.length == 0? "empty": (
-          znode.startsWith(zkw.assignmentZNode)?
-            ZKAssign.toString(data): // We should not be doing this reaching into another class
           znode.startsWith(zkw.metaServerZNode)?
             getServerNameOrEmptyString(data):
           znode.startsWith(zkw.backupMasterAddressesZNode)?

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 c16d6d3..c8afac3 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.zookeeper;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
@@ -92,8 +93,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
   public String backupMasterAddressesZNode;
   // znode containing the current cluster state
   public String clusterStateZNode;
-  // znode used for region transitioning and assignment
-  public String assignmentZNode;
   // znode used for table disabling/enabling
   public String tableZNode;
   // znode containing the unique cluster ID
@@ -111,11 +110,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
 
 
   // Certain ZooKeeper nodes need to be world-readable
-  public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
-    new ArrayList<ACL>() { {
-      add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
-      add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
-    }};
+  public static final List<ACL> CREATOR_ALL_AND_WORLD_READABLE =
+    Arrays.asList(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE),
+      new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
 
   private final Configuration conf;
 
@@ -171,9 +168,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     try {
       // Create all the necessary "directories" of znodes
       ZKUtil.createWithParents(this, baseZNode);
-      if (conf.getBoolean("hbase.assignment.usezk", false)) {
-        ZKUtil.createAndFailSilent(this, assignmentZNode);
-      }
       ZKUtil.createAndFailSilent(this, rsZNode);
       ZKUtil.createAndFailSilent(this, drainingZNode);
       ZKUtil.createAndFailSilent(this, tableZNode);
@@ -220,8 +214,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
         conf.get("zookeeper.znode.backup.masters", "backup-masters"));
     clusterStateZNode = ZKUtil.joinZNode(baseZNode,
         conf.get("zookeeper.znode.state", "running"));
-    assignmentZNode = ZKUtil.joinZNode(baseZNode,
-        conf.get("zookeeper.znode.unassigned", "region-in-transition"));
     tableZNode = ZKUtil.joinZNode(baseZNode,
         conf.get("zookeeper.znode.tableEnableDisable", "table"));
     clusterIdZNode = ZKUtil.joinZNode(baseZNode,


[09/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
index a1caf87..10274b4 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
@@ -2117,1093 +2117,6 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:ClusterUp)
   }
 
-  public interface RegionTransitionOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required uint32 event_type_code = 1;
-    /**
-     * <code>required uint32 event_type_code = 1;</code>
-     *
-     * <pre>
-     * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-     * </pre>
-     */
-    boolean hasEventTypeCode();
-    /**
-     * <code>required uint32 event_type_code = 1;</code>
-     *
-     * <pre>
-     * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-     * </pre>
-     */
-    int getEventTypeCode();
-
-    // required bytes region_name = 2;
-    /**
-     * <code>required bytes region_name = 2;</code>
-     *
-     * <pre>
-     * Full regionname in bytes
-     * </pre>
-     */
-    boolean hasRegionName();
-    /**
-     * <code>required bytes region_name = 2;</code>
-     *
-     * <pre>
-     * Full regionname in bytes
-     * </pre>
-     */
-    com.google.protobuf.ByteString getRegionName();
-
-    // required uint64 create_time = 3;
-    /**
-     * <code>required uint64 create_time = 3;</code>
-     */
-    boolean hasCreateTime();
-    /**
-     * <code>required uint64 create_time = 3;</code>
-     */
-    long getCreateTime();
-
-    // required .ServerName server_name = 4;
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    boolean hasServerName();
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName();
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
-
-    // optional bytes payload = 5;
-    /**
-     * <code>optional bytes payload = 5;</code>
-     */
-    boolean hasPayload();
-    /**
-     * <code>optional bytes payload = 5;</code>
-     */
-    com.google.protobuf.ByteString getPayload();
-  }
-  /**
-   * Protobuf type {@code RegionTransition}
-   *
-   * <pre>
-   **
-   * What we write under unassigned up in zookeeper as a region moves through
-   * open/close, etc., regions.  Details a region in transition.
-   * </pre>
-   */
-  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: {
-              bitField0_ |= 0x00000001;
-              eventTypeCode_ = input.readUInt32();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              regionName_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              createTime_ = input.readUInt64();
-              break;
-            }
-            case 34: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000008) == 0x00000008)) {
-                subBuilder = serverName_.toBuilder();
-              }
-              serverName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(serverName_);
-                serverName_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000008;
-              break;
-            }
-            case 42: {
-              bitField0_ |= 0x00000010;
-              payload_ = input.readBytes();
-              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 {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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;
-    }
-
-    private int bitField0_;
-    // required uint32 event_type_code = 1;
-    public static final int EVENT_TYPE_CODE_FIELD_NUMBER = 1;
-    private int eventTypeCode_;
-    /**
-     * <code>required uint32 event_type_code = 1;</code>
-     *
-     * <pre>
-     * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-     * </pre>
-     */
-    public boolean hasEventTypeCode() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required uint32 event_type_code = 1;</code>
-     *
-     * <pre>
-     * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-     * </pre>
-     */
-    public int getEventTypeCode() {
-      return eventTypeCode_;
-    }
-
-    // required bytes region_name = 2;
-    public static final int REGION_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString regionName_;
-    /**
-     * <code>required bytes region_name = 2;</code>
-     *
-     * <pre>
-     * Full regionname in bytes
-     * </pre>
-     */
-    public boolean hasRegionName() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required bytes region_name = 2;</code>
-     *
-     * <pre>
-     * Full regionname in bytes
-     * </pre>
-     */
-    public com.google.protobuf.ByteString getRegionName() {
-      return regionName_;
-    }
-
-    // required uint64 create_time = 3;
-    public static final int CREATE_TIME_FIELD_NUMBER = 3;
-    private long createTime_;
-    /**
-     * <code>required uint64 create_time = 3;</code>
-     */
-    public boolean hasCreateTime() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>required uint64 create_time = 3;</code>
-     */
-    public long getCreateTime() {
-      return createTime_;
-    }
-
-    // required .ServerName server_name = 4;
-    public static final int SERVER_NAME_FIELD_NUMBER = 4;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_;
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    public boolean hasServerName() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
-      return serverName_;
-    }
-    /**
-     * <code>required .ServerName server_name = 4;</code>
-     *
-     * <pre>
-     * The region server where the transition will happen or is happening
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
-      return serverName_;
-    }
-
-    // optional bytes payload = 5;
-    public static final int PAYLOAD_FIELD_NUMBER = 5;
-    private com.google.protobuf.ByteString payload_;
-    /**
-     * <code>optional bytes payload = 5;</code>
-     */
-    public boolean hasPayload() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional bytes payload = 5;</code>
-     */
-    public com.google.protobuf.ByteString getPayload() {
-      return payload_;
-    }
-
-    private void initFields() {
-      eventTypeCode_ = 0;
-      regionName_ = com.google.protobuf.ByteString.EMPTY;
-      createTime_ = 0L;
-      serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
-      payload_ = com.google.protobuf.ByteString.EMPTY;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasEventTypeCode()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasRegionName()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasCreateTime()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasServerName()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getServerName().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.writeUInt32(1, eventTypeCode_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, regionName_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt64(3, createTime_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeMessage(4, serverName_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(5, payload_);
-      }
-      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
-          .computeUInt32Size(1, eventTypeCode_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, regionName_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(3, createTime_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, serverName_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(5, payload_);
-      }
-      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.ZooKeeperProtos.RegionTransition)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition) obj;
-
-      boolean result = true;
-      result = result && (hasEventTypeCode() == other.hasEventTypeCode());
-      if (hasEventTypeCode()) {
-        result = result && (getEventTypeCode()
-            == other.getEventTypeCode());
-      }
-      result = result && (hasRegionName() == other.hasRegionName());
-      if (hasRegionName()) {
-        result = result && getRegionName()
-            .equals(other.getRegionName());
-      }
-      result = result && (hasCreateTime() == other.hasCreateTime());
-      if (hasCreateTime()) {
-        result = result && (getCreateTime()
-            == other.getCreateTime());
-      }
-      result = result && (hasServerName() == other.hasServerName());
-      if (hasServerName()) {
-        result = result && getServerName()
-            .equals(other.getServerName());
-      }
-      result = result && (hasPayload() == other.hasPayload());
-      if (hasPayload()) {
-        result = result && getPayload()
-            .equals(other.getPayload());
-      }
-      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 (hasEventTypeCode()) {
-        hash = (37 * hash) + EVENT_TYPE_CODE_FIELD_NUMBER;
-        hash = (53 * hash) + getEventTypeCode();
-      }
-      if (hasRegionName()) {
-        hash = (37 * hash) + REGION_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionName().hashCode();
-      }
-      if (hasCreateTime()) {
-        hash = (37 * hash) + CREATE_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getCreateTime());
-      }
-      if (hasServerName()) {
-        hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getServerName().hashCode();
-      }
-      if (hasPayload()) {
-        hash = (37 * hash) + PAYLOAD_FIELD_NUMBER;
-        hash = (53 * hash) + getPayload().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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.ZooKeeperProtos.RegionTransition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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.ZooKeeperProtos.RegionTransition parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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.ZooKeeperProtos.RegionTransition parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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.ZooKeeperProtos.RegionTransition parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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.ZooKeeperProtos.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}
-     *
-     * <pre>
-     **
-     * What we write under unassigned up in zookeeper as a region moves through
-     * open/close, etc., regions.  Details a region in transition.
-     * </pre>
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransitionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.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) {
-          getServerNameFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        eventTypeCode_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        createTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        if (serverNameBuilder_ == null) {
-          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
-        } else {
-          serverNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000008);
-        payload_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_RegionTransition_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition build() {
-        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.eventTypeCode_ = eventTypeCode_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.regionName_ = regionName_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.createTime_ = createTime_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        if (serverNameBuilder_ == null) {
-          result.serverName_ = serverName_;
-        } else {
-          result.serverName_ = serverNameBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.payload_ = payload_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition.getDefaultInstance()) return this;
-        if (other.hasEventTypeCode()) {
-          setEventTypeCode(other.getEventTypeCode());
-        }
-        if (other.hasRegionName()) {
-          setRegionName(other.getRegionName());
-        }
-        if (other.hasCreateTime()) {
-          setCreateTime(other.getCreateTime());
-        }
-        if (other.hasServerName()) {
-          mergeServerName(other.getServerName());
-        }
-        if (other.hasPayload()) {
-          setPayload(other.getPayload());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasEventTypeCode()) {
-          
-          return false;
-        }
-        if (!hasRegionName()) {
-          
-          return false;
-        }
-        if (!hasCreateTime()) {
-          
-          return false;
-        }
-        if (!hasServerName()) {
-          
-          return false;
-        }
-        if (!getServerName().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.ZooKeeperProtos.RegionTransition parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionTransition) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required uint32 event_type_code = 1;
-      private int eventTypeCode_ ;
-      /**
-       * <code>required uint32 event_type_code = 1;</code>
-       *
-       * <pre>
-       * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-       * </pre>
-       */
-      public boolean hasEventTypeCode() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required uint32 event_type_code = 1;</code>
-       *
-       * <pre>
-       * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-       * </pre>
-       */
-      public int getEventTypeCode() {
-        return eventTypeCode_;
-      }
-      /**
-       * <code>required uint32 event_type_code = 1;</code>
-       *
-       * <pre>
-       * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-       * </pre>
-       */
-      public Builder setEventTypeCode(int value) {
-        bitField0_ |= 0x00000001;
-        eventTypeCode_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint32 event_type_code = 1;</code>
-       *
-       * <pre>
-       * Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-       * </pre>
-       */
-      public Builder clearEventTypeCode() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        eventTypeCode_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // required bytes region_name = 2;
-      private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>required bytes region_name = 2;</code>
-       *
-       * <pre>
-       * Full regionname in bytes
-       * </pre>
-       */
-      public boolean hasRegionName() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required bytes region_name = 2;</code>
-       *
-       * <pre>
-       * Full regionname in bytes
-       * </pre>
-       */
-      public com.google.protobuf.ByteString getRegionName() {
-        return regionName_;
-      }
-      /**
-       * <code>required bytes region_name = 2;</code>
-       *
-       * <pre>
-       * Full regionname in bytes
-       * </pre>
-       */
-      public Builder setRegionName(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        regionName_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bytes region_name = 2;</code>
-       *
-       * <pre>
-       * Full regionname in bytes
-       * </pre>
-       */
-      public Builder clearRegionName() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        regionName_ = getDefaultInstance().getRegionName();
-        onChanged();
-        return this;
-      }
-
-      // required uint64 create_time = 3;
-      private long createTime_ ;
-      /**
-       * <code>required uint64 create_time = 3;</code>
-       */
-      public boolean hasCreateTime() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>required uint64 create_time = 3;</code>
-       */
-      public long getCreateTime() {
-        return createTime_;
-      }
-      /**
-       * <code>required uint64 create_time = 3;</code>
-       */
-      public Builder setCreateTime(long value) {
-        bitField0_ |= 0x00000004;
-        createTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint64 create_time = 3;</code>
-       */
-      public Builder clearCreateTime() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        createTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // required .ServerName server_name = 4;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName serverName_ = 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> serverNameBuilder_;
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public boolean hasServerName() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServerName() {
-        if (serverNameBuilder_ == null) {
-          return serverName_;
-        } else {
-          return serverNameBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public Builder setServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
-        if (serverNameBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          serverName_ = value;
-          onChanged();
-        } else {
-          serverNameBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000008;
-        return this;
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public Builder setServerName(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
-        if (serverNameBuilder_ == null) {
-          serverName_ = builderForValue.build();
-          onChanged();
-        } else {
-          serverNameBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000008;
-        return this;
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public Builder mergeServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
-        if (serverNameBuilder_ == null) {
-          if (((bitField0_ & 0x00000008) == 0x00000008) &&
-              serverName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
-            serverName_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(serverName_).mergeFrom(value).buildPartial();
-          } else {
-            serverName_ = value;
-          }
-          onChanged();
-        } else {
-          serverNameBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000008;
-        return this;
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public Builder clearServerName() {
-        if (serverNameBuilder_ == null) {
-          serverName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
-          onChanged();
-        } else {
-          serverNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000008);
-        return this;
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerNameBuilder() {
-        bitField0_ |= 0x00000008;
-        onChanged();
-        return getServerNameFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </pre>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
-        if (serverNameBuilder_ != null) {
-          return serverNameBuilder_.getMessageOrBuilder();
-        } else {
-          return serverName_;
-        }
-      }
-      /**
-       * <code>required .ServerName server_name = 4;</code>
-       *
-       * <pre>
-       * The region server where the transition will happen or is happening
-       * </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> 
-          getServerNameFieldBuilder() {
-        if (serverNameBuilder_ == null) {
-          serverNameBuilder_ = 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>(
-                  serverName_,
-                  getParentForChildren(),
-                  isClean());
-          serverName_ = null;
-        }
-        return serverNameBuilder_;
-      }
-
-      // optional bytes payload = 5;
-      private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>optional bytes payload = 5;</code>
-       */
-      public boolean hasPayload() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional bytes payload = 5;</code>
-       */
-      public com.google.protobuf.ByteString getPayload() {
-        return payload_;
-      }
-      /**
-       * <code>optional bytes payload = 5;</code>
-       */
-      public Builder setPayload(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000010;
-        payload_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bytes payload = 5;</code>
-       */
-      public Builder clearPayload() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        payload_ = getDefaultInstance().getPayload();
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:RegionTransition)
-    }
-
-    static {
-      defaultInstance = new RegionTransition(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:RegionTransition)
-  }
-
   public interface SplitLogTaskOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -10448,11 +9361,6 @@ public final class ZooKeeperProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_ClusterUp_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_RegionTransition_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_RegionTransition_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_SplitLogTask_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -10510,38 +9418,35 @@ public final class ZooKeeperProtos {
       "gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerName\022" +
       "\023\n\013rpc_version\030\002 \001(\r\":\n\006Master\022\033\n\006master" +
       "\030\001 \002(\0132\013.ServerName\022\023\n\013rpc_version\030\002 \001(\r" +
-      "\"\037\n\tClusterUp\022\022\n\nstart_date\030\001 \002(\t\"\210\001\n\020Re" +
-      "gionTransition\022\027\n\017event_type_code\030\001 \002(\r\022" +
-      "\023\n\013region_name\030\002 \002(\014\022\023\n\013create_time\030\003 \002(" +
-      "\004\022 \n\013server_name\030\004 \002(\0132\013.ServerName\022\017\n\007p" +
-      "ayload\030\005 \001(\014\"\214\002\n\014SplitLogTask\022\"\n\005state\030\001" +
-      " \002(\0162\023.SplitLogTask.State\022 \n\013server_name",
-      "\030\002 \002(\0132\013.ServerName\0221\n\004mode\030\003 \001(\0162\032.Spli" +
-      "tLogTask.RecoveryMode:\007UNKNOWN\"C\n\005State\022" +
-      "\016\n\nUNASSIGNED\020\000\022\t\n\005OWNED\020\001\022\014\n\010RESIGNED\020\002" +
-      "\022\010\n\004DONE\020\003\022\007\n\003ERR\020\004\">\n\014RecoveryMode\022\013\n\007U" +
-      "NKNOWN\020\000\022\021\n\rLOG_SPLITTING\020\001\022\016\n\nLOG_REPLA" +
-      "Y\020\002\"n\n\005Table\022$\n\005state\030\001 \002(\0162\014.Table.Stat" +
-      "e:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISA" +
-      "BLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"\215\001\n\017" +
-      "ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r" +
-      "eplicationEndpointImpl\030\002 \001(\t\022\035\n\004data\030\003 \003",
-      "(\0132\017.BytesBytesPair\022&\n\rconfiguration\030\004 \003" +
-      "(\0132\017.NameStringPair\"^\n\020ReplicationState\022" +
-      "&\n\005state\030\001 \002(\0162\027.ReplicationState.State\"" +
-      "\"\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027R" +
-      "eplicationHLogPosition\022\020\n\010position\030\001 \002(\003" +
-      "\"%\n\017ReplicationLock\022\022\n\nlock_owner\030\001 \002(\t\"" +
-      "\230\001\n\tTableLock\022\036\n\ntable_name\030\001 \001(\0132\n.Tabl" +
-      "eName\022\037\n\nlock_owner\030\002 \001(\0132\013.ServerName\022\021" +
-      "\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022\017\n\007" +
-      "purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\";\n\017St",
-      "oreSequenceId\022\023\n\013family_name\030\001 \002(\014\022\023\n\013se" +
-      "quence_id\030\002 \002(\004\"g\n\026RegionStoreSequenceId" +
-      "s\022 \n\030last_flushed_sequence_id\030\001 \002(\004\022+\n\021s" +
-      "tore_sequence_id\030\002 \003(\0132\020.StoreSequenceId" +
-      "BE\n*org.apache.hadoop.hbase.protobuf.gen" +
-      "eratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
+      "\"\037\n\tClusterUp\022\022\n\nstart_date\030\001 \002(\t\"\214\002\n\014Sp" +
+      "litLogTask\022\"\n\005state\030\001 \002(\0162\023.SplitLogTask" +
+      ".State\022 \n\013server_name\030\002 \002(\0132\013.ServerName" +
+      "\0221\n\004mode\030\003 \001(\0162\032.SplitLogTask.RecoveryMo" +
+      "de:\007UNKNOWN\"C\n\005State\022\016\n\nUNASSIGNED\020\000\022\t\n\005" +
+      "OWNED\020\001\022\014\n\010RESIGNED\020\002\022\010\n\004DONE\020\003\022\007\n\003ERR\020\004",
+      "\">\n\014RecoveryMode\022\013\n\007UNKNOWN\020\000\022\021\n\rLOG_SPL" +
+      "ITTING\020\001\022\016\n\nLOG_REPLAY\020\002\"n\n\005Table\022$\n\005sta" +
+      "te\030\001 \002(\0162\014.Table.State:\007ENABLED\"?\n\005State" +
+      "\022\013\n\007ENABLED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING" +
+      "\020\002\022\014\n\010ENABLING\020\003\"\215\001\n\017ReplicationPeer\022\022\n\n" +
+      "clusterkey\030\001 \002(\t\022\037\n\027replicationEndpointI" +
+      "mpl\030\002 \001(\t\022\035\n\004data\030\003 \003(\0132\017.BytesBytesPair" +
+      "\022&\n\rconfiguration\030\004 \003(\0132\017.NameStringPair" +
+      "\"^\n\020ReplicationState\022&\n\005state\030\001 \002(\0162\027.Re" +
+      "plicationState.State\"\"\n\005State\022\013\n\007ENABLED",
+      "\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHLogPosit" +
+      "ion\022\020\n\010position\030\001 \002(\003\"%\n\017ReplicationLock" +
+      "\022\022\n\nlock_owner\030\001 \002(\t\"\230\001\n\tTableLock\022\036\n\nta" +
+      "ble_name\030\001 \001(\0132\n.TableName\022\037\n\nlock_owner" +
+      "\030\002 \001(\0132\013.ServerName\022\021\n\tthread_id\030\003 \001(\003\022\021" +
+      "\n\tis_shared\030\004 \001(\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013cr" +
+      "eate_time\030\006 \001(\003\";\n\017StoreSequenceId\022\023\n\013fa" +
+      "mily_name\030\001 \002(\014\022\023\n\013sequence_id\030\002 \002(\004\"g\n\026" +
+      "RegionStoreSequenceIds\022 \n\030last_flushed_s" +
+      "equence_id\030\001 \002(\004\022+\n\021store_sequence_id\030\002 ",
+      "\003(\0132\020.StoreSequenceIdBE\n*org.apache.hado" +
+      "op.hbase.protobuf.generatedB\017ZooKeeperPr" +
+      "otosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -10566,62 +9471,56 @@ public final class ZooKeeperProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ClusterUp_descriptor,
               new java.lang.String[] { "StartDate", });
-          internal_static_RegionTransition_descriptor =
-            getDescriptor().getMessageTypes().get(3);
-          internal_static_RegionTransition_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_RegionTransition_descriptor,
-              new java.lang.String[] { "EventTypeCode", "RegionName", "CreateTime", "ServerName", "Payload", });
           internal_static_SplitLogTask_descriptor =
-            getDescriptor().getMessageTypes().get(4);
+            getDescriptor().getMessageTypes().get(3);
           internal_static_SplitLogTask_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_SplitLogTask_descriptor,
               new java.lang.String[] { "State", "ServerName", "Mode", });
           internal_static_Table_descriptor =
-            getDescriptor().getMessageTypes().get(5);
+            getDescriptor().getMessageTypes().get(4);
           internal_static_Table_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_Table_descriptor,
               new java.lang.String[] { "State", });
           internal_static_ReplicationPeer_descriptor =
-            getDescriptor().getMessageTypes().get(6);
+            getDescriptor().getMessageTypes().get(5);
           internal_static_ReplicationPeer_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ReplicationPeer_descriptor,
               new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", });
           internal_static_ReplicationState_descriptor =
-            getDescriptor().getMessageTypes().get(7);
+            getDescriptor().getMessageTypes().get(6);
           internal_static_ReplicationState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ReplicationState_descriptor,
               new java.lang.String[] { "State", });
           internal_static_ReplicationHLogPosition_descriptor =
-            getDescriptor().getMessageTypes().get(8);
+            getDescriptor().getMessageTypes().get(7);
           internal_static_ReplicationHLogPosition_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ReplicationHLogPosition_descriptor,
               new java.lang.String[] { "Position", });
           internal_static_ReplicationLock_descriptor =
-            getDescriptor().getMessageTypes().get(9);
+            getDescriptor().getMessageTypes().get(8);
           internal_static_ReplicationLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ReplicationLock_descriptor,
               new java.lang.String[] { "LockOwner", });
           internal_static_TableLock_descriptor =
-            getDescriptor().getMessageTypes().get(10);
+            getDescriptor().getMessageTypes().get(9);
           internal_static_TableLock_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_TableLock_descriptor,
               new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
           internal_static_StoreSequenceId_descriptor =
-            getDescriptor().getMessageTypes().get(11);
+            getDescriptor().getMessageTypes().get(10);
           internal_static_StoreSequenceId_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_StoreSequenceId_descriptor,
               new java.lang.String[] { "FamilyName", "SequenceId", });
           internal_static_RegionStoreSequenceIds_descriptor =
-            getDescriptor().getMessageTypes().get(12);
+            getDescriptor().getMessageTypes().get(11);
           internal_static_RegionStoreSequenceIds_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_RegionStoreSequenceIds_descriptor,

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-protocol/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
index 598385c..4d727c6 100644
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
@@ -59,21 +59,6 @@ message ClusterUp {
 }
 
 /**
- * What we write under unassigned up in zookeeper as a region moves through
- * open/close, etc., regions.  Details a region in transition.
- */
-message RegionTransition {
-  // Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
-  required uint32 event_type_code = 1;
-  // Full regionname in bytes
-  required bytes region_name = 2;
-  required uint64 create_time = 3;
-  // The region server where the transition will happen or is happening
-  required ServerName server_name = 4;
-  optional bytes payload = 5;
-}
-
-/**
  * WAL SplitLog directory znodes have this for content.  Used doing distributed
  * WAL splitting.  Holds current state and name of server that originated split.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
index 9c9bfba..1891941 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
@@ -52,24 +52,4 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
   @Override
   public abstract TableStateManager getTableStateManager() throws InterruptedException,
     CoordinatedStateException;
-
-  /**
-   * Method to retrieve coordination for split transaction.
-   */
-  abstract public SplitTransactionCoordination getSplitTransactionCoordination();
-
-  /**
-   * Method to retrieve coordination for closing region operations.
-   */
-  public abstract CloseRegionCoordination getCloseRegionCoordination();
-
-  /**
-   * Method to retrieve coordination for opening region operations.
-   */
-  public abstract OpenRegionCoordination getOpenRegionCoordination();
-
-  /**
-   * Method to retrieve coordination for region merge transaction
-   */
-  public abstract  RegionMergeCoordination getRegionMergeCoordination();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/CloseRegionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/CloseRegionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/CloseRegionCoordination.java
deleted file mode 100644
index 503e4fc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/CloseRegionCoordination.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * 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.coordination;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-
-/**
- * Coordinated operations for close region handlers.
- */
-@InterfaceAudience.Private
-public interface CloseRegionCoordination {
-
-  /**
-   * Called before actual region closing to check that we can do close operation
-   * on this region.
-   * @param regionInfo region being closed
-   * @param crd details about closing operation
-   * @return true if caller shall proceed and close, false if need to abort closing.
-   */
-  boolean checkClosingState(HRegionInfo regionInfo, CloseRegionDetails crd);
-
-  /**
-   * Called after region is closed to notify all interesting parties / "register"
-   * region as finally closed.
-   * @param region region being closed
-   * @param sn ServerName on which task runs
-   * @param crd details about closing operation
-   */
-  void setClosedState(HRegion region, ServerName sn, CloseRegionDetails crd);
-
-  /**
-   * Construct CloseRegionDetails instance from CloseRegionRequest.
-   * @return instance of CloseRegionDetails
-   */
-  CloseRegionDetails parseFromProtoRequest(AdminProtos.CloseRegionRequest request);
-
-  /**
-   * Get details object with params for case when we're closing on
-   * regionserver side internally (not because of RPC call from master),
-   * so we don't parse details from protobuf request.
-   */
-  CloseRegionDetails getDetaultDetails();
-
-  /**
-   * Marker interface for region closing tasks. Used to carry implementation details in
-   * encapsulated way through Handlers to the consensus API.
-   */
-  static interface CloseRegionDetails {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/OpenRegionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/OpenRegionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/OpenRegionCoordination.java
deleted file mode 100644
index 0c6871d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/OpenRegionCoordination.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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.coordination;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-
-import java.io.IOException;
-
-/**
- * Cocoordination operations for opening regions.
- */
-@InterfaceAudience.Private
-public interface OpenRegionCoordination {
-
-  //---------------------
-  // RS-side operations
-  //---------------------
-  /**
-   * Tries to move regions to OPENED state.
-   *
-   * @param r Region we're working on.
-   * @param ord details about region opening task
-   * @return whether transition was successful or not
-   * @throws java.io.IOException
-   */
-  boolean transitionToOpened(HRegion r, OpenRegionDetails ord) throws IOException;
-
-  /**
-   * Transitions region from offline to opening state.
-   * @param regionInfo region we're working on.
-   * @param ord details about opening task.
-   * @return true if successful, false otherwise
-   */
-  boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
-                                         OpenRegionDetails ord);
-
-  /**
-   * Heartbeats to prevent timeouts.
-   *
-   * @param ord details about opening task.
-   * @param regionInfo region we're working on.
-   * @param rsServices instance of RegionServerrServices
-   * @param context used for logging purposes only
-   * @return true if successful heartbeat, false otherwise.
-   */
-  boolean tickleOpening(OpenRegionDetails ord, HRegionInfo regionInfo,
-                        RegionServerServices rsServices, String context);
-
-  /**
-   * Tries transition region from offline to failed open.
-   * @param rsServices instance of RegionServerServices
-   * @param hri region we're working on
-   * @param ord details about region opening task
-   * @return true if successful, false otherwise
-   */
-  boolean tryTransitionFromOfflineToFailedOpen(RegionServerServices rsServices,
-                                               HRegionInfo hri, OpenRegionDetails ord);
-
-  /**
-   * Tries transition from Opening to Failed open.
-   * @param hri region we're working on
-   * @param ord details about region opening task
-   * @return true if successfu. false otherwise.
-   */
-  boolean tryTransitionFromOpeningToFailedOpen(HRegionInfo hri, OpenRegionDetails ord);
-
-  /**
-   * Construct OpenRegionDetails instance from part of protobuf request.
-   * @return instance of OpenRegionDetails.
-   */
-  OpenRegionDetails parseFromProtoRequest(AdminProtos.OpenRegionRequest.RegionOpenInfo
-                                            regionOpenInfo);
-
-  /**
-   * Get details object with params for case when we're opening on
-   * regionserver side with all "default" properties.
-   */
-  OpenRegionDetails getDetailsForNonCoordinatedOpening();
-
-  //-------------------------
-  // HMaster-side operations
-  //-------------------------
-
-  /**
-   * Commits opening operation on HM side (steps required for "commit"
-   * are determined by coordination implementation).
-   * @return true if committed successfully, false otherwise.
-   */
-  public boolean commitOpenOnMasterSide(AssignmentManager assignmentManager,
-                                        HRegionInfo regionInfo,
-                                        OpenRegionDetails ord);
-
-  /**
-   * Interface for region opening tasks. Used to carry implementation details in
-   * encapsulated way through Handlers to the coordination API.
-   */
-  static interface OpenRegionDetails {
-    /**
-     * Sets server name on which opening operation is running.
-     */
-    void setServerName(ServerName serverName);
-
-    /**
-     * @return server name on which opening op is running.
-     */
-    ServerName getServerName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/RegionMergeCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/RegionMergeCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/RegionMergeCoordination.java
deleted file mode 100644
index b51dd9c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/RegionMergeCoordination.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * 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.coordination;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-
-/**
- * Coordination operations for region merge transaction. The operation should be coordinated at the
- * following stages:<br>
- * 1. startRegionMergeTransaction - all preparation/initialization for merge region transaction<br>
- * 2. waitForRegionMergeTransaction - wait until coordination complete all works related 
- * to merge<br>
- * 3. confirmRegionMergeTransaction - confirm that the merge could be completed and none of merging
- * regions moved somehow<br>
- * 4. completeRegionMergeTransaction - all steps that are required to complete the transaction.
- * Called after PONR (point of no return) <br>
- */
-@InterfaceAudience.Private
-public interface RegionMergeCoordination {
-
-  RegionMergeDetails getDefaultDetails();
-
-  /**
-   * Dummy interface for region merge transaction details.
-   */
-  public static interface RegionMergeDetails {
-  }
-
-  /**
-   * Start the region merge transaction
-   * @param region region to be created as offline
-   * @param serverName server event originates from
-   * @throws IOException
-   */
-  void startRegionMergeTransaction(HRegionInfo region, ServerName serverName, HRegionInfo a,
-      HRegionInfo b) throws IOException;
-
-  /**
-   * Get everything ready for region merge
-   * @throws IOException
-   */
-  void waitForRegionMergeTransaction(RegionServerServices services, HRegionInfo mergedRegionInfo,
-      HRegion region_a, HRegion region_b, RegionMergeDetails details) throws IOException;
-
-  /**
-   * Confirm that the region merge can be performed
-   * @param merged region
-   * @param a merging region A
-   * @param b merging region B
-   * @param serverName server event originates from
-   * @param rmd region merge details
-   * @throws IOException If thrown, transaction failed.
-   */
-  void confirmRegionMergeTransaction(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
-      ServerName serverName, RegionMergeDetails rmd) throws IOException;
-
-  /**
-   * @param merged region
-   * @param a merging region A
-   * @param b merging region B
-   * @param serverName server event originates from
-   * @param rmd region merge details
-   * @throws IOException
-   */
-  void processRegionMergeRequest(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
-      ServerName serverName, RegionMergeDetails rmd) throws IOException;
-
-  /**
-   * Finish off merge transaction
-   * @param services Used to online/offline regions.
-   * @param merged region
-   * @param region_a merging region A
-   * @param region_b merging region B
-   * @param rmd region merge details
-   * @param mergedRegion
-   * @throws IOException If thrown, transaction failed. Call
-   *           {@link RegionMergeTransaction#rollback(Server, RegionServerServices)}
-   */
-  void completeRegionMergeTransaction(RegionServerServices services, HRegionInfo merged,
-      HRegion region_a, HRegion region_b, RegionMergeDetails rmd, HRegion mergedRegion)
-      throws IOException;
-
-  /**
-   * This method is used during rollback
-   * @param merged region to be rolled back
-   */
-  void clean(HRegionInfo merged);
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitTransactionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitTransactionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitTransactionCoordination.java
deleted file mode 100644
index 659d4e5..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitTransactionCoordination.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- *
- * 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.coordination;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.SplitTransaction;
-
-/**
- * Coordination operations for split transaction. The split operation should be coordinated at the
- * following stages:
- * 1. start - all preparation/initialization for split transaction should be done there.
- * 2. waitForSplitTransaction  - the coordination should perform all logic related to split
- *    transaction and wait till it's finished
- * 3. completeSplitTransaction - all steps that are required to complete the transaction.
- *    Called after PONR (point of no return)
- */
-@InterfaceAudience.Private
-public interface SplitTransactionCoordination {
-
-  /**
-   * Dummy interface for split transaction details.
-   */
-  public static interface SplitTransactionDetails {
-  }
-
-  SplitTransactionDetails getDefaultDetails();
-
-
-  /**
-   * init coordination for split transaction
-   * @param parent region to be created as offline
-   * @param serverName server event originates from
-   * @param hri_a daughter region
-   * @param hri_b daughter region
-   * @throws IOException
-   */
-  void startSplitTransaction(HRegion parent, ServerName serverName,
-      HRegionInfo hri_a, HRegionInfo hri_b) throws IOException;
-
-  /**
-   * Wait while coordination process the transaction
-   * @param services Used to online/offline regions.
-   * @param parent region
-   * @param hri_a daughter region
-   * @param hri_b daughter region
-   * @param std split transaction details
-   * @throws IOException
-   */
-  void waitForSplitTransaction(final RegionServerServices services,
-      HRegion parent, HRegionInfo hri_a, HRegionInfo hri_b, SplitTransactionDetails std)
-      throws IOException;
-
-  /**
-   * Finish off split transaction
-   * @param services Used to online/offline regions.
-   * @param first daughter region
-   * @param second daughter region
-   * @param std split transaction details
-   * @param parent
-   * @throws IOException If thrown, transaction failed. Call
-   *           {@link SplitTransaction#rollback(Server, RegionServerServices)}
-   */
-  void completeSplitTransaction(RegionServerServices services, HRegion first,
-      HRegion second, SplitTransactionDetails std, HRegion parent) throws IOException;
-
-  /**
-   * clean the split transaction
-   * @param hri node to delete
-   */
-  void clean(final HRegionInfo hri);
-
-  /**
-   * Required by AssignmentManager
-   */
-  int processTransition(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b,
-      ServerName sn, SplitTransactionDetails std) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java
deleted file mode 100644
index de9f51f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * 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.coordination;
-
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLIT;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLITTING;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REQUEST_REGION_SPLIT;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.SplitTransaction;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-
-public class ZKSplitTransactionCoordination implements SplitTransactionCoordination {
-
-  private CoordinatedStateManager coordinationManager;
-  private final ZooKeeperWatcher watcher;
-
-  private static final Log LOG = LogFactory.getLog(ZKSplitTransactionCoordination.class);
-
-  public ZKSplitTransactionCoordination(CoordinatedStateManager coordinationProvider,
-      ZooKeeperWatcher watcher) {
-    this.coordinationManager = coordinationProvider;
-    this.watcher = watcher;
-  }
-
-  /**
-   * Creates a new ephemeral node in the PENDING_SPLIT state for the specified region. Create it
-   * ephemeral in case regionserver dies mid-split.
-   * <p>
-   * Does not transition nodes from other states. If a node already exists for this region, an
-   * Exception will be thrown.
-   * @param parent region to be created as offline
-   * @param serverName server event originates from
-   * @param hri_a daughter region
-   * @param hri_b daughter region
-   * @throws IOException
-   */
-
-  @Override
-  public void startSplitTransaction(HRegion parent, ServerName serverName, HRegionInfo hri_a,
-      HRegionInfo hri_b) throws IOException {
-
-    HRegionInfo region = parent.getRegionInfo();
-    try {
-
-      LOG.debug(watcher.prefix("Creating ephemeral node for " + region.getEncodedName()
-          + " in PENDING_SPLIT state"));
-      byte[] payload = HRegionInfo.toDelimitedByteArray(hri_a, hri_b);
-      RegionTransition rt =
-          RegionTransition.createRegionTransition(RS_ZK_REQUEST_REGION_SPLIT,
-            region.getRegionName(), serverName, payload);
-      String node = ZKAssign.getNodeName(watcher, region.getEncodedName());
-      if (!ZKUtil.createEphemeralNodeAndWatch(watcher, node, rt.toByteArray())) {
-        throw new IOException("Failed create of ephemeral " + node);
-      }
-
-    } catch (KeeperException e) {
-      throw new IOException("Failed creating PENDING_SPLIT znode on "
-          + parent.getRegionNameAsString(), e);
-    }
-
-  }
-
-  /**
-   * Transitions an existing ephemeral node for the specified region which is currently in the begin
-   * state to be in the end state. Master cleans up the final SPLIT znode when it reads it (or if we
-   * crash, zk will clean it up).
-   * <p>
-   * Does not transition nodes from other states. If for some reason the node could not be
-   * transitioned, the method returns -1. If the transition is successful, the version of the node
-   * after transition is returned.
-   * <p>
-   * This method can fail and return false for three different reasons:
-   * <ul>
-   * <li>Node for this region does not exist</li>
-   * <li>Node for this region is not in the begin state</li>
-   * <li>After verifying the begin state, update fails because of wrong version (this should never
-   * actually happen since an RS only does this transition following a transition to the begin
-   * state. If two RS are conflicting, one would fail the original transition to the begin state and
-   * not this transition)</li>
-   * </ul>
-   * <p>
-   * Does not set any watches.
-   * <p>
-   * This method should only be used by a RegionServer when splitting a region.
-   * @param parent region to be transitioned to opened
-   * @param a Daughter a of split
-   * @param b Daughter b of split
-   * @param serverName server event originates from
-   * @param std split transaction details
-   * @param beginState the expected current state the znode should be
-   * @param endState the state to be transition to
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws IOException
-   */
-
-  private int transitionSplittingNode(HRegionInfo parent, HRegionInfo a, HRegionInfo b,
-      ServerName serverName, SplitTransactionDetails std, final EventType beginState,
-      final EventType endState) throws IOException {
-    ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) std;
-    byte[] payload = HRegionInfo.toDelimitedByteArray(a, b);
-    try {
-      return ZKAssign.transitionNode(watcher, parent, serverName, beginState, endState,
-        zstd.getZnodeVersion(), payload);
-    } catch (KeeperException e) {
-      throw new IOException(
-          "Failed transition of splitting node " + parent.getRegionNameAsString(), e);
-    }
-  }
-
-  /**
-   * Wait for the splitting node to be transitioned from pending_split to splitting by master.
-   * That's how we are sure master has processed the event and is good with us to move on. If we
-   * don't get any update, we periodically transition the node so that master gets the callback. If
-   * the node is removed or is not in pending_split state any more, we abort the split.
-   */
-  @Override
-  public void waitForSplitTransaction(final RegionServerServices services, HRegion parent,
-      HRegionInfo hri_a, HRegionInfo hri_b, SplitTransactionDetails sptd) throws IOException {
-    ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) sptd;
-
-    // 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.
-    try {
-      int spins = 0;
-      Stat stat = new Stat();
-      ServerName expectedServer = coordinationManager.getServer().getServerName();
-      String node = parent.getRegionInfo().getEncodedName();
-      while (!(coordinationManager.getServer().isStopped() || services.isStopping())) {
-        if (spins % 5 == 0) {
-          LOG.debug("Still waiting for master to process " + "the pending_split for " + node);
-          SplitTransactionDetails temp = getDefaultDetails();
-          transitionSplittingNode(parent.getRegionInfo(), hri_a, hri_b, expectedServer, temp,
-            RS_ZK_REQUEST_REGION_SPLIT, RS_ZK_REQUEST_REGION_SPLIT);
-        }
-        Thread.sleep(100);
-        spins++;
-        byte[] data = ZKAssign.getDataNoWatch(watcher, node, stat);
-        if (data == null) {
-          throw new IOException("Data is null, splitting node " + node + " no longer exists");
-        }
-        RegionTransition rt = RegionTransition.parseFrom(data);
-        EventType et = rt.getEventType();
-        if (et == RS_ZK_REGION_SPLITTING) {
-          ServerName serverName = rt.getServerName();
-          if (!serverName.equals(expectedServer)) {
-            throw new IOException("Splitting node " + node + " is for " + serverName + ", not us "
-                + expectedServer);
-          }
-          byte[] payloadOfSplitting = rt.getPayload();
-          List<HRegionInfo> splittingRegions =
-              HRegionInfo.parseDelimitedFrom(payloadOfSplitting, 0, payloadOfSplitting.length);
-          assert splittingRegions.size() == 2;
-          HRegionInfo a = splittingRegions.get(0);
-          HRegionInfo b = splittingRegions.get(1);
-          if (!(hri_a.equals(a) && hri_b.equals(b))) {
-            throw new IOException("Splitting node " + node + " is for " + a + ", " + b
-                + ", not expected daughters: " + hri_a + ", " + hri_b);
-          }
-          // Master has processed it.
-          zstd.setZnodeVersion(stat.getVersion());
-          return;
-        }
-        if (et != RS_ZK_REQUEST_REGION_SPLIT) {
-          throw new IOException("Splitting node " + node + " moved out of splitting to " + et);
-        }
-      }
-      // Server is stopping/stopped
-      throw new IOException("Server is " + (services.isStopping() ? "stopping" : "stopped"));
-    } catch (Exception e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IOException("Failed getting SPLITTING znode on " + parent.getRegionNameAsString(),
-          e);
-    }
-  }
-
-  /**
-   * Finish off split transaction, transition the zknode
-   * @param services Used to online/offline regions.
-   * @param a daughter region
-   * @param b daughter region
-   * @param std split transaction details
-   * @param parent
-   * @throws IOException If thrown, transaction failed. Call
-   *           {@link SplitTransaction#rollback(Server, RegionServerServices)}
-   */
-  @Override
-  public void completeSplitTransaction(final RegionServerServices services, HRegion a, HRegion b,
-      SplitTransactionDetails std, HRegion parent) throws IOException {
-    ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) std;
-    // Tell master about split by updating zk. If we fail, abort.
-    if (coordinationManager.getServer() != null) {
-      try {
-        zstd.setZnodeVersion(transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(),
-          b.getRegionInfo(), coordinationManager.getServer().getServerName(), zstd,
-          RS_ZK_REGION_SPLITTING, RS_ZK_REGION_SPLIT));
-
-        int spins = 0;
-        // Now wait for the master to process the split. 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 split for "
-                + parent.getRegionInfo().getEncodedName());
-          }
-          Thread.sleep(100);
-          // When this returns -1 it means the znode doesn't exist
-          zstd.setZnodeVersion(transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(),
-            b.getRegionInfo(), coordinationManager.getServer().getServerName(), zstd,
-            RS_ZK_REGION_SPLIT, RS_ZK_REGION_SPLIT));
-          spins++;
-        } while (zstd.getZnodeVersion() != -1 && !coordinationManager.getServer().isStopped()
-            && !services.isStopping());
-      } catch (Exception e) {
-        if (e instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        throw new IOException("Failed telling master about split", e);
-      }
-    }
-
-    // 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
-    // deleted and cleaned up.
-  }
-
-  @Override
-  public void clean(final HRegionInfo hri) {
-    try {
-      // Only delete if its in expected state; could have been hijacked.
-      if (!ZKAssign.deleteNode(coordinationManager.getServer().getZooKeeper(),
-        hri.getEncodedName(), RS_ZK_REQUEST_REGION_SPLIT, coordinationManager.getServer()
-            .getServerName())) {
-        ZKAssign.deleteNode(coordinationManager.getServer().getZooKeeper(), hri.getEncodedName(),
-          RS_ZK_REGION_SPLITTING, coordinationManager.getServer().getServerName());
-      }
-    } catch (KeeperException.NoNodeException e) {
-      LOG.info("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
-    } catch (KeeperException e) {
-      coordinationManager.getServer().abort("Failed cleanup of " + hri.getRegionNameAsString(), e);
-    }
-  }
-
-  /**
-   * ZK-based implementation. Has details about whether the state transition should be reflected in
-   * ZK, as well as expected version of znode.
-   */
-  public static class ZkSplitTransactionDetails implements
-      SplitTransactionCoordination.SplitTransactionDetails {
-    private int znodeVersion;
-
-    public ZkSplitTransactionDetails() {
-    }
-
-    /**
-     * @return znode current version
-     */
-    public int getZnodeVersion() {
-      return znodeVersion;
-    }
-
-    /**
-     * @param znodeVersion znode new version
-     */
-    public void setZnodeVersion(int znodeVersion) {
-      this.znodeVersion = znodeVersion;
-    }
-  }
-
-  @Override
-  public SplitTransactionDetails getDefaultDetails() {
-    ZkSplitTransactionDetails zstd = new ZkSplitTransactionDetails();
-    zstd.setZnodeVersion(-1);
-    return zstd;
-  }
-
-  @Override
-  public int processTransition(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b, ServerName sn,
-      SplitTransactionDetails std) throws IOException {
-    return transitionSplittingNode(p, hri_a, hri_b, sn, std, RS_ZK_REQUEST_REGION_SPLIT,
-      RS_ZK_REGION_SPLITTING);
-
-  }
-}


[08/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCloseRegionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCloseRegionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCloseRegionCoordination.java
deleted file mode 100644
index 77ed84d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCloseRegionCoordination.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/**
- * 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.coordination;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.IOException;
-
-/**
- * ZK-based implementation of {@link CloseRegionCoordination}.
- */
-@InterfaceAudience.Private
-public class ZkCloseRegionCoordination implements CloseRegionCoordination {
-  private static final Log LOG = LogFactory.getLog(ZkCloseRegionCoordination.class);
-
-  private final static int FAILED_VERSION = -1;
-
-  private CoordinatedStateManager csm;
-  private final ZooKeeperWatcher watcher;
-
-  public ZkCloseRegionCoordination(CoordinatedStateManager csm, ZooKeeperWatcher watcher) {
-    this.csm = csm;
-    this.watcher = watcher;
-  }
-
-  /**
-   * In ZK-based version we're checking for bad znode state, e.g. if we're
-   * trying to delete the znode, and it's not ours (version doesn't match).
-   */
-  @Override
-  public boolean checkClosingState(HRegionInfo regionInfo, CloseRegionDetails crd) {
-    ZkCloseRegionDetails zkCrd = (ZkCloseRegionDetails) crd;
-
-    try {
-      return zkCrd.isPublishStatusInZk() && !ZKAssign.checkClosingState(watcher,
-        regionInfo, ((ZkCloseRegionDetails) crd).getExpectedVersion());
-    } catch (KeeperException ke) {
-       csm.getServer().abort("Unrecoverable exception while checking state with zk " +
-          regionInfo.getRegionNameAsString() + ", still finishing close", ke);
-        throw new RuntimeException(ke);
-    }
-  }
-
-  /**
-   * In ZK-based version we do some znodes transitioning.
-   */
-  @Override
-  public void setClosedState(HRegion region, ServerName sn, CloseRegionDetails crd) {
-    ZkCloseRegionDetails zkCrd = (ZkCloseRegionDetails) crd;
-    String name = region.getRegionInfo().getRegionNameAsString();
-
-    if (zkCrd.isPublishStatusInZk()) {
-      if (setClosedState(region,sn, zkCrd)) {
-        LOG.debug("Set closed state in zk for " + name + " on " + sn);
-      } else {
-        LOG.debug("Set closed state in zk UNSUCCESSFUL for " + name + " on " + sn);
-      }
-    }
-  }
-
-  /**
-   * Parse ZK-related fields from request.
-   */
-  @Override
-  public CloseRegionDetails parseFromProtoRequest(AdminProtos.CloseRegionRequest request) {
-    ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
-      new ZkCloseRegionCoordination.ZkCloseRegionDetails();
-    zkCrd.setPublishStatusInZk(request.getTransitionInZK());
-    int versionOfClosingNode = -1;
-    if (request.hasVersionOfClosingNode()) {
-      versionOfClosingNode = request.getVersionOfClosingNode();
-    }
-    zkCrd.setExpectedVersion(versionOfClosingNode);
-
-    return zkCrd;
-  }
-
-  /**
-   * No ZK tracking will be performed for that case.
-   * This method should be used when we want to construct CloseRegionDetails,
-   * but don't want any coordination on that (when it's initiated by regionserver),
-   * so no znode state transitions will be performed.
-   */
-  @Override
-  public CloseRegionDetails getDetaultDetails() {
-    ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
-      new ZkCloseRegionCoordination.ZkCloseRegionDetails();
-    zkCrd.setPublishStatusInZk(false);
-    zkCrd.setExpectedVersion(FAILED_VERSION);
-
-    return zkCrd;
-  }
-
-  /**
-   * Transition ZK node to CLOSED
-   * @param region HRegion instance being closed
-   * @param sn ServerName on which task runs
-   * @param zkCrd  details about region closing operation.
-   * @return If the state is set successfully
-   */
-  private boolean setClosedState(final HRegion region,
-                                 ServerName sn,
-                                 ZkCloseRegionDetails zkCrd) {
-    final int expectedVersion = zkCrd.getExpectedVersion();
-
-    try {
-      if (ZKAssign.transitionNodeClosed(watcher, region.getRegionInfo(),
-        sn, expectedVersion) == FAILED_VERSION) {
-        LOG.warn("Completed the CLOSE of a region but when transitioning from " +
-          " CLOSING to CLOSED got a version mismatch, someone else clashed " +
-          "so now unassigning");
-        region.close();
-        return false;
-      }
-    } catch (NullPointerException e) {
-      // I've seen NPE when table was deleted while close was running in unit tests.
-      LOG.warn("NPE during close -- catching and continuing...", e);
-      return false;
-    } catch (KeeperException e) {
-      LOG.error("Failed transitioning node from CLOSING to CLOSED", e);
-      return false;
-    } catch (IOException e) {
-      LOG.error("Failed to close region after failing to transition", e);
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * ZK-based implementation. Has details about whether the state transition should be
-   * reflected in ZK, as well as expected version of znode.
-   */
-  public static class ZkCloseRegionDetails implements CloseRegionCoordination.CloseRegionDetails {
-
-    /**
-     * True if we are to update zk about the region close; if the close
-     * was orchestrated by master, then update zk.  If the close is being run by
-     * the regionserver because its going down, don't update zk.
-     * */
-    private boolean publishStatusInZk;
-
-    /**
-     * The version of znode to compare when RS transitions the znode from
-     * CLOSING state.
-     */
-    private int expectedVersion = FAILED_VERSION;
-
-    public ZkCloseRegionDetails() {
-    }
-
-    public ZkCloseRegionDetails(boolean publishStatusInZk, int expectedVersion) {
-      this.publishStatusInZk = publishStatusInZk;
-      this.expectedVersion = expectedVersion;
-    }
-
-    public boolean isPublishStatusInZk() {
-      return publishStatusInZk;
-    }
-
-    public void setPublishStatusInZk(boolean publishStatusInZk) {
-      this.publishStatusInZk = publishStatusInZk;
-    }
-
-    public int getExpectedVersion() {
-      return expectedVersion;
-    }
-
-    public void setExpectedVersion(int expectedVersion) {
-      this.expectedVersion = expectedVersion;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index a5492a9..4d62e54 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.Server;
@@ -32,23 +30,13 @@ import org.apache.zookeeper.KeeperException;
  */
 @InterfaceAudience.Private
 public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
-  private static final Log LOG = LogFactory.getLog(ZkCoordinatedStateManager.class);
   protected Server server;
   protected ZooKeeperWatcher watcher;
-  protected SplitTransactionCoordination splitTransactionCoordination;
-  protected CloseRegionCoordination closeRegionCoordination;
-  protected OpenRegionCoordination openRegionCoordination;
-  protected RegionMergeCoordination regionMergeCoordination;
 
   @Override
   public void initialize(Server server) {
     this.server = server;
     this.watcher = server.getZooKeeper();
-
-    splitTransactionCoordination = new ZKSplitTransactionCoordination(this, watcher);
-    closeRegionCoordination = new ZkCloseRegionCoordination(this, watcher);
-    openRegionCoordination = new ZkOpenRegionCoordination(this, watcher);
-    regionMergeCoordination = new ZkRegionMergeCoordination(this, watcher);
   }
 
   @Override
@@ -65,24 +53,4 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
       throw new CoordinatedStateException(e);
     }
   }
-
-  @Override
-  public SplitTransactionCoordination getSplitTransactionCoordination() {
-    return splitTransactionCoordination;
-  }
-
-  @Override
-  public CloseRegionCoordination getCloseRegionCoordination() {
-    return closeRegionCoordination;
-  }
-
-  @Override
-  public OpenRegionCoordination getOpenRegionCoordination() {
-    return openRegionCoordination;
-  }
-
-  @Override
-  public RegionMergeCoordination getRegionMergeCoordination() {
-    return regionMergeCoordination;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java
deleted file mode 100644
index 290ed09..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * 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.coordination;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.IOException;
-
-/**
- * ZK-based implementation of {@link OpenRegionCoordination}.
- */
-@InterfaceAudience.Private
-public class ZkOpenRegionCoordination implements OpenRegionCoordination {
-  private static final Log LOG = LogFactory.getLog(ZkOpenRegionCoordination.class);
-
-  private CoordinatedStateManager coordination;
-  private final ZooKeeperWatcher watcher;
-
-  public ZkOpenRegionCoordination(CoordinatedStateManager coordination,
-                                  ZooKeeperWatcher watcher) {
-    this.coordination = coordination;
-    this.watcher = watcher;
-  }
-
-  //-------------------------------
-  // Region Server-side operations
-  //-------------------------------
-
-  /**
-   * @param r Region we're working on.
-   * @return whether znode is successfully transitioned to OPENED state.
-   * @throws java.io.IOException
-   */
-  @Override
-  public boolean transitionToOpened(final HRegion r, OpenRegionDetails ord) throws IOException {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-
-    boolean result = false;
-    HRegionInfo hri = r.getRegionInfo();
-    final String name = hri.getRegionNameAsString();
-    // Finally, Transition ZK node to OPENED
-    try {
-      if (ZKAssign.transitionNodeOpened(watcher, hri,
-        zkOrd.getServerName(), zkOrd.getVersion()) == -1) {
-        String warnMsg = "Completed the OPEN of region " + name +
-          " but when transitioning from " + " OPENING to OPENED ";
-        try {
-          String node = ZKAssign.getNodeName(watcher, hri.getEncodedName());
-          if (ZKUtil.checkExists(watcher, node) < 0) {
-            // if the znode
-            coordination.getServer().abort(warnMsg + "the znode disappeared", null);
-          } else {
-            LOG.warn(warnMsg + "got a version mismatch, someone else clashed; " +
-              "so now unassigning -- closing region on server: " + zkOrd.getServerName());
-          }
-        } catch (KeeperException ke) {
-          coordination.getServer().abort(warnMsg, ke);
-        }
-      } else {
-        LOG.debug("Transitioned " + r.getRegionInfo().getEncodedName() +
-          " to OPENED in zk on " + zkOrd.getServerName());
-        result = true;
-      }
-    } catch (KeeperException e) {
-      LOG.error("Failed transitioning node " + name +
-        " from OPENING to OPENED -- closing region", e);
-    }
-    return result;
-  }
-
-  /**
-   * Transition ZK node from OFFLINE to OPENING.
-   * @param regionInfo region info instance
-   * @param ord - instance of open region details, for ZK implementation
-   *   will include version Of OfflineNode that needs to be compared
-   *   before changing the node's state from OFFLINE
-   * @return True if successful transition.
-   */
-  @Override
-  public boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
-                                                OpenRegionDetails ord) {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-
-    // encoded name is used as znode encoded name in ZK
-    final String encodedName = regionInfo.getEncodedName();
-
-    // TODO: should also handle transition from CLOSED?
-    try {
-      // Initialize the znode version.
-      zkOrd.setVersion(ZKAssign.transitionNode(watcher, regionInfo,
-        zkOrd.getServerName(), EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, zkOrd.getVersionOfOfflineNode()));
-    } catch (KeeperException e) {
-      LOG.error("Error transition from OFFLINE to OPENING for region=" +
-        encodedName, e);
-      zkOrd.setVersion(-1);
-      return false;
-    }
-    boolean b = isGoodVersion(zkOrd);
-    if (!b) {
-      LOG.warn("Failed transition from OFFLINE to OPENING for region=" +
-        encodedName);
-    }
-    return b;
-  }
-
-  /**
-   * Update our OPENING state in zookeeper.
-   * Do this so master doesn't timeout this region-in-transition.
-   * 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.
-   *
-   * @param context Some context to add to logs if failure
-   * @return True if successful transition.
-   */
-  @Override
-  public boolean tickleOpening(OpenRegionDetails ord, HRegionInfo regionInfo,
-                               RegionServerServices rsServices, final String context) {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-    if (!isRegionStillOpening(regionInfo, rsServices)) {
-      LOG.warn("Open region aborted since it isn't opening any more");
-      return false;
-    }
-    // If previous checks failed... do not try again.
-    if (!isGoodVersion(zkOrd)) return false;
-    String encodedName = regionInfo.getEncodedName();
-    try {
-      zkOrd.setVersion(ZKAssign.confirmNodeOpening(watcher,
-          regionInfo, zkOrd.getServerName(), zkOrd.getVersion()));
-    } catch (KeeperException e) {
-      coordination.getServer().abort("Exception refreshing OPENING; region=" + encodedName +
-        ", context=" + context, e);
-      zkOrd.setVersion(-1);
-      return false;
-    }
-    boolean b = isGoodVersion(zkOrd);
-    if (!b) {
-      LOG.warn("Failed refreshing OPENING; region=" + encodedName +
-        ", context=" + context);
-    }
-    return b;
-  }
-
-  /**
-   * Try to transition to open.
-   *
-   * This is not guaranteed to succeed, we just do our best.
-   *
-   * @param rsServices
-   * @param hri Region we're working on.
-   * @param ord Details about region open task
-   * @return whether znode is successfully transitioned to FAILED_OPEN state.
-   */
-  @Override
-  public boolean tryTransitionFromOfflineToFailedOpen(RegionServerServices rsServices,
-                                                      final HRegionInfo hri,
-                                                      OpenRegionDetails ord) {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-    boolean result = false;
-    final String name = hri.getRegionNameAsString();
-    try {
-      LOG.info("Opening of region " + hri + " failed, transitioning" +
-        " from OFFLINE to FAILED_OPEN in ZK, expecting version " +
-        zkOrd.getVersionOfOfflineNode());
-      if (ZKAssign.transitionNode(
-        rsServices.getZooKeeper(), hri,
-        rsServices.getServerName(),
-        EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_FAILED_OPEN,
-        zkOrd.getVersionOfOfflineNode()) == -1) {
-        LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
-          "It's likely that the master already timed out this open " +
-          "attempt, and thus another RS already has the region.");
-      } else {
-        result = true;
-      }
-    } catch (KeeperException e) {
-      LOG.error("Failed transitioning node " + name + " from OFFLINE to FAILED_OPEN", e);
-    }
-    return result;
-  }
-
-  private boolean isGoodVersion(ZkOpenRegionDetails zkOrd) {
-    return zkOrd.getVersion() != -1;
-  }
-
-  /**
-   * This is not guaranteed to succeed, we just do our best.
-   * @param hri Region we're working on.
-   * @return whether znode is successfully transitioned to FAILED_OPEN state.
-   */
-  @Override
-  public boolean tryTransitionFromOpeningToFailedOpen(final HRegionInfo hri,
-                                                      OpenRegionDetails ord) {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-    boolean result = false;
-    final String name = hri.getRegionNameAsString();
-    try {
-      LOG.info("Opening of region " + hri + " failed, transitioning" +
-        " from OPENING to FAILED_OPEN in ZK, expecting version " + zkOrd.getVersion());
-      if (ZKAssign.transitionNode(
-        watcher, hri,
-        zkOrd.getServerName(),
-        EventType.RS_ZK_REGION_OPENING,
-        EventType.RS_ZK_REGION_FAILED_OPEN,
-        zkOrd.getVersion()) == -1) {
-        LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
-          "It's likely that the master already timed out this open " +
-          "attempt, and thus another RS already has the region.");
-      } else {
-        result = true;
-      }
-    } catch (KeeperException e) {
-      LOG.error("Failed transitioning node " + name +
-        " from OPENING to FAILED_OPEN", e);
-    }
-    return result;
-  }
-
-  /**
-   * Parse ZK-related fields from request.
-   */
-  @Override
-  public OpenRegionCoordination.OpenRegionDetails parseFromProtoRequest(
-      AdminProtos.OpenRegionRequest.RegionOpenInfo regionOpenInfo) {
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-
-    int versionOfOfflineNode = -1;
-    if (regionOpenInfo.hasVersionOfOfflineNode()) {
-      versionOfOfflineNode = regionOpenInfo.getVersionOfOfflineNode();
-    }
-    zkCrd.setVersionOfOfflineNode(versionOfOfflineNode);
-    zkCrd.setServerName(coordination.getServer().getServerName());
-
-    return zkCrd;
-  }
-
-  /**
-   * No ZK tracking will be performed for that case.
-   * This method should be used when we want to construct CloseRegionDetails,
-   * but don't want any coordination on that (when it's initiated by regionserver),
-   * so no znode state transitions will be performed.
-   */
-  @Override
-  public OpenRegionCoordination.OpenRegionDetails getDetailsForNonCoordinatedOpening() {
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setVersionOfOfflineNode(-1);
-    zkCrd.setServerName(coordination.getServer().getServerName());
-
-    return zkCrd;
-  }
-
-  //--------------------------
-  // HMaster-side operations
-  //--------------------------
-  @Override
-  public boolean commitOpenOnMasterSide(AssignmentManager assignmentManager,
-                                        HRegionInfo regionInfo,
-                                        OpenRegionDetails ord) {
-    boolean committedSuccessfully = true;
-
-    // Code to defend against case where we get SPLIT before region open
-    // processing completes; temporary till we make SPLITs go via zk -- 0.92.
-    RegionState regionState = assignmentManager.getRegionStates()
-      .getRegionTransitionState(regionInfo.getEncodedName());
-    boolean openedNodeDeleted = false;
-    if (regionState != null && regionState.isOpened()) {
-      openedNodeDeleted = deleteOpenedNode(regionInfo, ord);
-      if (!openedNodeDeleted) {
-        LOG.error("Znode of region " + regionInfo.getShortNameToLog() + " could not be deleted.");
-      }
-    } else {
-      LOG.warn("Skipping the onlining of " + regionInfo.getShortNameToLog() +
-        " because regions is NOT in RIT -- presuming this is because it SPLIT");
-    }
-    if (!openedNodeDeleted) {
-      if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(),
-          ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
-        debugLog(regionInfo, "Opened region "
-          + regionInfo.getShortNameToLog() + " but "
-          + "this table is disabled, triggering close of region");
-        committedSuccessfully = false;
-      }
-    }
-
-    return committedSuccessfully;
-  }
-
-  private boolean deleteOpenedNode(HRegionInfo regionInfo, OpenRegionDetails ord) {
-    ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
-    int expectedVersion = zkOrd.getVersion();
-
-    debugLog(regionInfo, "Handling OPENED of " +
-      regionInfo.getShortNameToLog() + " from " + zkOrd.getServerName().toString() +
-      "; deleting unassigned node");
-    try {
-      // delete the opened znode only if the version matches.
-      return ZKAssign.deleteNode(this.coordination.getServer().getZooKeeper(),
-        regionInfo.getEncodedName(), EventType.RS_ZK_REGION_OPENED, expectedVersion);
-    } catch(KeeperException.NoNodeException e){
-      // Getting no node exception here means that already the region has been opened.
-      LOG.warn("The znode of the region " + regionInfo.getShortNameToLog() +
-        " would have already been deleted");
-      return false;
-    } catch (KeeperException e) {
-      this.coordination.getServer().abort("Error deleting OPENED node in ZK (" +
-        regionInfo.getRegionNameAsString() + ")", e);
-    }
-    return false;
-  }
-
-  private void debugLog(HRegionInfo region, String string) {
-    if (region.isMetaTable()) {
-      LOG.info(string);
-    } else {
-      LOG.debug(string);
-    }
-  }
-
-  // Additional classes and helper methods
-
-  /**
-   * ZK-based implementation. Has details about whether the state transition should be
-   * reflected in ZK, as well as expected version of znode.
-   */
-  public static class ZkOpenRegionDetails implements OpenRegionCoordination.OpenRegionDetails {
-
-    // We get version of our znode at start of open process and monitor it across
-    // the total open. We'll fail the open if someone hijacks our znode; we can
-    // tell this has happened if version is not as expected.
-    private volatile int version = -1;
-
-    //version of the offline node that was set by the master
-    private volatile int versionOfOfflineNode = -1;
-
-    /**
-     * Server name the handler is running on.
-     */
-    private ServerName serverName;
-
-    public ZkOpenRegionDetails() {
-    }
-
-    public ZkOpenRegionDetails(int versionOfOfflineNode) {
-      this.versionOfOfflineNode = versionOfOfflineNode;
-    }
-
-    public int getVersionOfOfflineNode() {
-      return versionOfOfflineNode;
-    }
-
-    public void setVersionOfOfflineNode(int versionOfOfflineNode) {
-      this.versionOfOfflineNode = versionOfOfflineNode;
-    }
-
-    public int getVersion() {
-      return version;
-    }
-
-    public void setVersion(int version) {
-      this.version = version;
-    }
-
-    @Override
-    public ServerName getServerName() {
-      return serverName;
-    }
-
-    @Override
-    public void setServerName(ServerName serverName) {
-      this.serverName = serverName;
-    }
-  }
-
-  private 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
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkRegionMergeCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkRegionMergeCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkRegionMergeCoordination.java
deleted file mode 100644
index 8c18821..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkRegionMergeCoordination.java
+++ /dev/null
@@ -1,326 +0,0 @@
-/**
- *
- * 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.coordination;
-
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_MERGED;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_MERGING;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REQUEST_REGION_MERGE;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.data.Stat;
-
-public class ZkRegionMergeCoordination implements RegionMergeCoordination {
-
-  private CoordinatedStateManager manager;
-  private final ZooKeeperWatcher watcher;
-
-  private static final Log LOG = LogFactory.getLog(ZkRegionMergeCoordination.class);
-
-  public ZkRegionMergeCoordination(CoordinatedStateManager manager,
-      ZooKeeperWatcher watcher) {
-    this.manager = manager;
-    this.watcher = watcher;
-  }
-
-  /**
-   * ZK-based implementation. Has details about whether the state transition should be reflected in
-   * ZK, as well as expected version of znode.
-   */
-  public static class ZkRegionMergeDetails implements RegionMergeCoordination.RegionMergeDetails {
-    private int znodeVersion;
-
-    public ZkRegionMergeDetails() {
-    }
-
-    public int getZnodeVersion() {
-      return znodeVersion;
-    }
-
-    public void setZnodeVersion(int znodeVersion) {
-      this.znodeVersion = znodeVersion;
-    }
-  }
-
-  @Override
-  public RegionMergeDetails getDefaultDetails() {
-    ZkRegionMergeDetails zstd = new ZkRegionMergeDetails();
-    zstd.setZnodeVersion(-1);
-    return zstd;
-  }
-
-  /**
-   * Wait for the merging node to be transitioned from pending_merge
-   * to merging by master. That's how we are sure master has processed
-   * the event and is good with us to move on. If we don't get any update,
-   * we periodically transition the node so that master gets the callback.
-   * If the node is removed or is not in pending_merge state any more,
-   * we abort the merge.
-   * @throws IOException
-   */
-
-  @Override
-  public void waitForRegionMergeTransaction(RegionServerServices services,
-      HRegionInfo mergedRegionInfo, HRegion region_a, HRegion region_b, RegionMergeDetails details)
-      throws IOException {
-    try {
-      int spins = 0;
-      Stat stat = new Stat();
-      ServerName expectedServer = manager.getServer().getServerName();
-      String node = mergedRegionInfo.getEncodedName();
-      ZkRegionMergeDetails zdetails = (ZkRegionMergeDetails) details;
-      while (!(manager.getServer().isStopped() || services.isStopping())) {
-        if (spins % 5 == 0) {
-          LOG.debug("Still waiting for master to process " + "the pending_merge for " + node);
-          ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) getDefaultDetails();
-          transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(),
-            region_b.getRegionInfo(), expectedServer, zrmd, RS_ZK_REQUEST_REGION_MERGE,
-            RS_ZK_REQUEST_REGION_MERGE);
-        }
-        Thread.sleep(100);
-        spins++;
-        byte[] data = ZKAssign.getDataNoWatch(watcher, node, stat);
-        if (data == null) {
-          throw new IOException("Data is null, merging node " + node + " no longer exists");
-        }
-        RegionTransition rt = RegionTransition.parseFrom(data);
-        EventType et = rt.getEventType();
-        if (et == RS_ZK_REGION_MERGING) {
-          ServerName serverName = rt.getServerName();
-          if (!serverName.equals(expectedServer)) {
-            throw new IOException("Merging node " + node + " is for " + serverName + ", not us "
-                + expectedServer);
-          }
-          byte[] payloadOfMerging = rt.getPayload();
-          List<HRegionInfo> mergingRegions =
-              HRegionInfo.parseDelimitedFrom(payloadOfMerging, 0, payloadOfMerging.length);
-          assert mergingRegions.size() == 3;
-          HRegionInfo a = mergingRegions.get(1);
-          HRegionInfo b = mergingRegions.get(2);
-          HRegionInfo hri_a = region_a.getRegionInfo();
-          HRegionInfo hri_b = region_b.getRegionInfo();
-          if (!(hri_a.equals(a) && hri_b.equals(b))) {
-            throw new IOException("Merging node " + node + " is for " + a + ", " + b
-                + ", not expected regions: " + hri_a + ", " + hri_b);
-          }
-          // Master has processed it.
-          zdetails.setZnodeVersion(stat.getVersion());
-          return;
-        }
-        if (et != RS_ZK_REQUEST_REGION_MERGE) {
-          throw new IOException("Merging node " + node + " moved out of merging to " + et);
-        }
-      }
-      // Server is stopping/stopped
-      throw new IOException("Server is " + (services.isStopping() ? "stopping" : "stopped"));
-    } catch (Exception e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IOException("Failed getting MERGING znode on "
-          + mergedRegionInfo.getRegionNameAsString(), e);
-    }
-  }
-
-  /**
-   * Creates a new ephemeral node in the PENDING_MERGE state for the merged region.
-   * Create it ephemeral in case regionserver dies mid-merge.
-   *
-   * <p>
-   * Does not transition nodes from other states. If a node already exists for
-   * this region, a {@link NodeExistsException} will be thrown.
-   *
-   * @param region region to be created as offline
-   * @param serverName server event originates from
-   * @throws IOException
-   */
-  @Override
-  public void startRegionMergeTransaction(final HRegionInfo region, final ServerName serverName,
-      final HRegionInfo a, final HRegionInfo b) throws IOException {
-    LOG.debug(watcher.prefix("Creating ephemeral node for " + region.getEncodedName()
-        + " in PENDING_MERGE state"));
-    byte[] payload = HRegionInfo.toDelimitedByteArray(region, a, b);
-    RegionTransition rt =
-        RegionTransition.createRegionTransition(RS_ZK_REQUEST_REGION_MERGE, region.getRegionName(),
-          serverName, payload);
-    String node = ZKAssign.getNodeName(watcher, region.getEncodedName());
-    try {
-      if (!ZKUtil.createEphemeralNodeAndWatch(watcher, node, rt.toByteArray())) {
-        throw new IOException("Failed create of ephemeral " + node);
-      }
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /*
-   * (non-Javadoc)
-   * @see
-   * org.apache.hadoop.hbase.regionserver.coordination.RegionMergeCoordination#clean(org.apache.hadoop
-   * .hbase.Server, org.apache.hadoop.hbase.HRegionInfo)
-   */
-  @Override
-  public void clean(final HRegionInfo hri) {
-    try {
-      // Only delete if its in expected state; could have been hijacked.
-      if (!ZKAssign.deleteNode(watcher, hri.getEncodedName(), RS_ZK_REQUEST_REGION_MERGE, manager
-          .getServer().getServerName())) {
-        ZKAssign.deleteNode(watcher, hri.getEncodedName(), RS_ZK_REGION_MERGING, manager
-            .getServer().getServerName());
-      }
-    } catch (KeeperException.NoNodeException e) {
-      LOG.info("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
-    } catch (KeeperException e) {
-      manager.getServer().abort("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
-    }
-  }
-
-  /*
-   * ZooKeeper implementation of finishRegionMergeTransaction
-   */
-  @Override
-  public void completeRegionMergeTransaction(final RegionServerServices services,
-      HRegionInfo mergedRegionInfo, HRegion region_a, HRegion region_b, RegionMergeDetails rmd,
-      HRegion mergedRegion) throws IOException {
-    ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) rmd;
-    if (manager.getServer() == null
-        || manager.getServer().getCoordinatedStateManager() == null) {
-      return;
-    }
-    // Tell master about merge by updating zk. If we fail, abort.
-    try {
-      transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo(),
-        manager.getServer().getServerName(), rmd, 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 "
-              + mergedRegionInfo.getEncodedName() + ", waited "
-              + (EnvironmentEdgeManager.currentTimeMillis() - startTime) + "ms");
-        }
-        Thread.sleep(100);
-        // When this returns -1 it means the znode doesn't exist
-        transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo(),
-          manager.getServer().getServerName(), rmd, RS_ZK_REGION_MERGED, RS_ZK_REGION_MERGED);
-        spins++;
-      } while (zrmd.getZnodeVersion() != -1 && !manager.getServer().isStopped()
-          && !services.isStopping());
-    } catch (Exception e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IOException("Failed telling master about merge "
-          + mergedRegionInfo.getEncodedName(), e);
-    }
-    // Leaving here, the mergedir with its dross will be in place but since the
-    // merge was successful, just leave it; it'll be cleaned when region_a is
-    // cleaned up by CatalogJanitor on master
-  }
-
-  /*
-   * Zookeeper implementation of region merge confirmation
-   */
-  @Override
-  public void confirmRegionMergeTransaction(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
-      ServerName serverName, RegionMergeDetails rmd) throws IOException {
-    transitionMergingNode(merged, a, b, serverName, rmd, RS_ZK_REGION_MERGING,
-      RS_ZK_REGION_MERGING);
-  }
-
-  /*
-   * Zookeeper implementation of region merge processing
-   */
-  @Override
-  public void processRegionMergeRequest(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b,
-      ServerName sn, RegionMergeDetails rmd) throws IOException {
-    transitionMergingNode(p, hri_a, hri_b, sn, rmd, EventType.RS_ZK_REQUEST_REGION_MERGE,
-      EventType.RS_ZK_REGION_MERGING);
-  }
-
-  /**
-   * Transitions an existing ephemeral node for the specified region which is
-   * currently in the begin state to be in the end state. Master cleans up the
-   * final MERGE znode when it reads it (or if we crash, zk will clean it up).
-   *
-   * <p>
-   * Does not transition nodes from other states. If for some reason the node
-   * could not be transitioned, the method returns -1. If the transition is
-   * successful, the version of the node after transition is updated in details.
-   *
-   * <p>
-   * This method can fail and return false for three different reasons:
-   * <ul>
-   * <li>Node for this region does not exist</li>
-   * <li>Node for this region is not in the begin state</li>
-   * <li>After verifying the begin state, update fails because of wrong version
-   * (this should never actually happen since an RS only does this transition
-   * following a transition to the begin state. If two RS are conflicting, one would
-   * fail the original transition to the begin state and not this transition)</li>
-   * </ul>
-   *
-   * <p>
-   * Does not set any watches.
-   *
-   * <p>
-   * This method should only be used by a RegionServer when merging two regions.
-   *
-   * @param merged region to be transitioned to opened
-   * @param a merging region A
-   * @param b merging region B
-   * @param serverName server event originates from
-   * @param rmd region merge details
-   * @param beginState the expected current state the node should be
-   * @param endState the state to be transition to
-   * @throws IOException
-   */
-  private void transitionMergingNode(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
-      ServerName serverName, RegionMergeDetails rmd, final EventType beginState,
-      final EventType endState) throws IOException {
-    ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) rmd;
-    byte[] payload = HRegionInfo.toDelimitedByteArray(merged, a, b);
-    try {
-      zrmd.setZnodeVersion(ZKAssign.transitionNode(watcher, merged, serverName, beginState,
-        endState, zrmd.getZnodeVersion(), payload));
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 ddab430..88e5c2a 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
@@ -45,7 +45,7 @@ public class AssignCallable implements Callable<Object> {
 
   @Override
   public Object call() throws Exception {
-    assignmentManager.assign(hri, true, newPlan);
+    assignmentManager.assign(hri, newPlan);
     return null;
   }
 }


[06/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 f8e99f1..cb301b9 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
@@ -258,12 +258,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * {@link #finishActiveMasterInitialization(MonitoredTask)} after
    * the master becomes the active one.
    *
-   * @throws InterruptedException
    * @throws KeeperException
    * @throws IOException
    */
   public HMaster(final Configuration conf, CoordinatedStateManager csm)
-      throws IOException, KeeperException, InterruptedException {
+      throws IOException, KeeperException {
     super(conf, csm);
     this.rsFatals = new MemoryBoundedLogMessageBuffer(
       conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
@@ -413,7 +412,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     this.assignmentManager = new AssignmentManager(this, serverManager,
       this.balancer, this.service, this.metricsMaster,
       this.tableLockManager);
-    zooKeeper.registerListenerFirst(assignmentManager);
 
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
         this.serverManager);
@@ -674,34 +672,29 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
     RegionStates regionStates = assignmentManager.getRegionStates();
     regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
-    boolean rit = this.assignmentManager
-      .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
     boolean metaRegionLocation = metaTableLocator.verifyMetaRegionLocation(
       this.getShortCircuitConnection(), this.getZooKeeper(), timeout);
     ServerName currentMetaServer = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
     if (!metaRegionLocation) {
       // Meta location is not verified. It should be in transition, or offline.
       // We will wait for it to be assigned in enableSSHandWaitForMeta below.
-      assigned++;
-      if (!rit) {
-        // Assign meta since not already in transition
-        if (currentMetaServer != null) {
-          // If the meta server is not known to be dead or online,
-          // just split the meta log, and don't expire it since this
-          // could be a full cluster restart. Otherwise, we will think
-          // this is a failover and lose previous region locations.
-          // If it is really a failover case, AM will find out in rebuilding
-          // user regions. Otherwise, we are good since all logs are split
-          // or known to be replayed before user regions are assigned.
-          if (serverManager.isServerOnline(currentMetaServer)) {
-            LOG.info("Forcing expire of " + currentMetaServer);
-            serverManager.expireServer(currentMetaServer);
-          }
-          splitMetaLogBeforeAssignment(currentMetaServer);
-          previouslyFailedMetaRSs.add(currentMetaServer);
+      if (currentMetaServer != null) {
+        // If the meta server is not known to be dead or online,
+        // just split the meta log, and don't expire it since this
+        // could be a full cluster restart. Otherwise, we will think
+        // this is a failover and lose previous region locations.
+        // If it is really a failover case, AM will find out in rebuilding
+        // user regions. Otherwise, we are good since all logs are split
+        // or known to be replayed before user regions are assigned.
+        if (serverManager.isServerOnline(currentMetaServer)) {
+          LOG.info("Forcing expire of " + currentMetaServer);
+          serverManager.expireServer(currentMetaServer);
         }
-        assignmentManager.assignMeta();
+        splitMetaLogBeforeAssignment(currentMetaServer);
+        previouslyFailedMetaRSs.add(currentMetaServer);
       }
+      assignmentManager.assignMeta();
+      assigned++;
     } else {
       // Region already assigned. We didn't assign it. Add to in-memory state.
       regionStates.updateRegionState(
@@ -725,8 +718,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     // No need to wait for meta is assigned = 0 when meta is just verified.
     enableServerShutdownHandler(assigned != 0);
 
-    LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
-      ", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
+    LOG.info("hbase:meta assigned=" + assigned + ", location="
+      + metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
     status.setStatus("META assigned.");
   }
 
@@ -1736,7 +1729,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   public void assignRegion(HRegionInfo hri) {
-    assignmentManager.assign(hri, true);
+    assignmentManager.assign(hri);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index d4f3d6d..d6f825b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -359,7 +359,7 @@ public class MasterRpcServices extends RSRpcServices
       }
       LOG.info(master.getClientIdAuditPrefix()
         + " assign " + regionInfo.getRegionNameAsString());
-      master.assignmentManager.assign(regionInfo, true, true);
+      master.assignmentManager.assign(regionInfo, true);
       if (master.cpHost != null) {
         master.cpHost.postAssign(regionInfo);
       }
@@ -1074,6 +1074,7 @@ public class MasterRpcServices extends RSRpcServices
    *
    */
   @Override
+  @SuppressWarnings("deprecation")
   public OfflineRegionResponse offlineRegion(RpcController controller,
       OfflineRegionRequest request) throws ServiceException {
     final byte [] regionName = request.getRegion().getValue().toByteArray();
@@ -1203,6 +1204,7 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  @SuppressWarnings("deprecation")
   public UnassignRegionResponse unassignRegion(RpcController controller,
       UnassignRegionRequest req) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OfflineCallback.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OfflineCallback.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OfflineCallback.java
deleted file mode 100644
index c93dbe1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OfflineCallback.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * 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.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.AsyncCallback.StringCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-
-/**
- * Callback handler for creating unassigned offline znodes
- * used during bulk assign, async setting region to offline.
- */
-@InterfaceAudience.Private
-public class OfflineCallback implements StringCallback {
-  private static final Log LOG = LogFactory.getLog(OfflineCallback.class);
-  private final ExistCallback callBack;
-  private final ZooKeeperWatcher zkw;
-  private final ServerName destination;
-  private final AtomicInteger counter;
-
-  OfflineCallback(final ZooKeeperWatcher zkw,
-      final ServerName destination, final AtomicInteger counter,
-      final Map<String, Integer> offlineNodesVersions) {
-    this.callBack = new ExistCallback(
-      destination, counter, offlineNodesVersions);
-    this.destination = destination;
-    this.counter = counter;
-    this.zkw = zkw;
-  }
-
-  @Override
-  public void processResult(int rc, String path, Object ctx, String name) {
-    if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
-      LOG.warn("Node for " + path + " already exists");
-    } else if (rc != 0) {
-      // This is result code.  If non-zero, need to resubmit.
-      LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
-        "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
-      this.counter.addAndGet(1);
-      return;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("rs=" + ctx + ", server=" + destination);
-    }
-    // Async exists to set a watcher so we'll get triggered when
-    // unassigned node changes.
-    ZooKeeper zk = this.zkw.getRecoverableZooKeeper().getZooKeeper();
-    zk.exists(path, this.zkw, callBack, ctx);
-  }
-
-  /**
-   * Callback handler for the exists call that sets watcher on unassigned znodes.
-   * Used during bulk assign on startup.
-   */
-  static class ExistCallback implements StatCallback {
-    private static final Log LOG = LogFactory.getLog(ExistCallback.class);
-    private final Map<String, Integer> offlineNodesVersions;
-    private final AtomicInteger counter;
-    private ServerName destination;
-
-    ExistCallback(final ServerName destination,
-        final AtomicInteger counter,
-        final Map<String, Integer> offlineNodesVersions) {
-      this.offlineNodesVersions = offlineNodesVersions;
-      this.destination = destination;
-      this.counter = counter;
-    }
-
-    @Override
-    public void processResult(int rc, String path, Object ctx, Stat stat) {
-      if (rc != 0) {
-        // This is result code.  If non-zero, need to resubmit.
-        LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
-          "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
-        this.counter.addAndGet(1);
-        return;
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("rs=" + ctx + ", server=" + destination);
-      }
-      HRegionInfo region = ((RegionState)ctx).getRegion();
-      offlineNodesVersions.put(
-        region.getEncodedName(), Integer.valueOf(stat.getVersion()));
-      this.counter.addAndGet(1);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 8e1e040..0e6e69e 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
@@ -22,16 +22,15 @@ 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.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
@@ -40,7 +39,6 @@ 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;
 
@@ -59,7 +57,6 @@ public class RegionStateStore {
   private volatile HTableInterface metaTable;
   private volatile boolean initialized;
 
-  private final boolean noPersistence;
   private final Server server;
 
   /**
@@ -131,25 +128,19 @@ public class RegionStateStore {
   }
 
   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);
     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,
-          server.getShortCircuitConnection());
-      }
+    if (server instanceof RegionServerServices) {
+      metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
+        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    }
+    if (metaRegion == null) {
+      metaTable = new HTable(TableName.META_TABLE_NAME,
+        server.getShortCircuitConnection());
     }
     initialized = true;
   }
@@ -170,7 +161,7 @@ public class RegionStateStore {
   @SuppressWarnings("deprecation")
   void updateRegionState(long openSeqNum,
       RegionState newState, RegionState oldState) {
-    if (noPersistence || !initialized) {
+    if (!initialized) {
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 67eda4a..f111107 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
@@ -34,21 +34,17 @@ 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableStateManager;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -351,28 +347,6 @@ public class RegionStates {
 
   /**
    * Update a region state. It will be put in transition if not already there.
-   *
-   * If we can't find the region info based on the region name in
-   * the transition, log a warning and return null.
-   */
-  public RegionState updateRegionState(
-      final RegionTransition transition, final State state) {
-    byte [] regionName = transition.getRegionName();
-    HRegionInfo regionInfo = getRegionInfo(regionName);
-    if (regionInfo == null) {
-      String prettyRegionName = HRegionInfo.prettyPrint(
-        HRegionInfo.encodeRegionName(regionName));
-      LOG.warn("Failed to find region " + prettyRegionName
-        + " in updating its state to " + state
-        + " based on region transition " + transition);
-      return null;
-    }
-    return updateRegionState(regionInfo, state,
-      transition.getServerName());
-  }
-
-  /**
-   * Update a region state. It will be put in transition if not already there.
    */
   public RegionState updateRegionState(
       final HRegionInfo hri, final State state, final ServerName serverName) {
@@ -548,8 +522,7 @@ public class RegionStates {
   /**
    * A server is offline, all regions on it are dead.
    */
-  public synchronized List<HRegionInfo> serverOffline(
-      final ZooKeeperWatcher watcher, final ServerName sn) {
+  public synchronized List<HRegionInfo> serverOffline(final ServerName sn) {
     // Offline all regions on this server not already in transition.
     List<HRegionInfo> rits = new ArrayList<HRegionInfo>();
     Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
@@ -565,13 +538,7 @@ public class RegionStates {
         regionsToOffline.add(region);
       } else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
         LOG.debug("Offline splitting/merging region " + getRegionState(region));
-        try {
-          // Delete the ZNode if exists
-          ZKAssign.deleteNodeFailSilent(watcher, region);
-          regionsToOffline.add(region);
-        } catch (KeeperException ke) {
-          server.abort("Unexpected ZK exception deleting node " + region, ke);
-        }
+        regionsToOffline.add(region);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 6204206..9390eba 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
@@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.R
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Triple;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -159,7 +159,7 @@ public class ServerManager {
    * handler is not enabled, is queued up.
    * <p>
    * So this is a set of region servers known to be dead but not submitted to
-   * ServerShutdownHander for processing yet.
+   * ServerShutdownHandler for processing yet.
    */
   private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
 
@@ -310,7 +310,7 @@ public class ServerManager {
    * Check is a server of same host and port already exists,
    * if not, or the existed one got a smaller start code, record it.
    *
-   * @param sn the server to check and record
+   * @param serverName the server to check and record
    * @param sl the server load on the server
    * @return true if the server is recorded, otherwise, false
    */
@@ -717,12 +717,10 @@ public class ServerManager {
    * <p>
    * @param server server to open a region
    * @param region region to open
-   * @param versionOfOfflineNode that needs to be present in the offline node
-   * when RS tries to change the state from OFFLINE to other states.
    * @param favoredNodes
    */
   public RegionOpeningState sendRegionOpen(final ServerName server,
-      HRegionInfo region, int versionOfOfflineNode, List<ServerName> favoredNodes)
+      HRegionInfo region, List<ServerName> favoredNodes)
   throws IOException {
     AdminService.BlockingInterface admin = getRsAdmin(server);
     if (admin == null) {
@@ -730,8 +728,8 @@ public class ServerManager {
         " failed because no RPC connection found to this server");
       return RegionOpeningState.FAILED_OPENING;
     }
-    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server, 
-      region, versionOfOfflineNode, favoredNodes, 
+    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
+      region, favoredNodes,
       (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
     try {
       OpenRegionResponse response = admin.openRegion(null, request);
@@ -751,7 +749,7 @@ public class ServerManager {
    * @return a list of region opening states
    */
   public List<RegionOpeningState> sendRegionOpen(ServerName server,
-      List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos)
+      List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos)
   throws IOException {
     AdminService.BlockingInterface admin = getRsAdmin(server);
     if (admin == null) {
@@ -760,7 +758,7 @@ public class ServerManager {
       return null;
     }
 
-    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(regionOpenInfos, 
+    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(regionOpenInfos,
       (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
     try {
       OpenRegionResponse response = admin.openRegion(null, request);
@@ -777,15 +775,11 @@ public class ServerManager {
    * have the specified region or the region is being split.
    * @param server server to open a region
    * @param region region to open
-   * @param versionOfClosingNode
-   *   the version of znode to compare when RS transitions the znode from
-   *   CLOSING state.
    * @param dest - if the region is moved to another server, the destination server. null otherwise.
-   * @return true if server acknowledged close, false if not
    * @throws IOException
    */
   public boolean sendRegionClose(ServerName server, HRegionInfo region,
-    int versionOfClosingNode, ServerName dest, boolean transitionInZK) throws IOException {
+      ServerName dest) throws IOException {
     if (server == null) throw new NullPointerException("Passed server is null");
     AdminService.BlockingInterface admin = getRsAdmin(server);
     if (admin == null) {
@@ -795,12 +789,12 @@ public class ServerManager {
         " failed because no RPC connection found to this server");
     }
     return ProtobufUtil.closeRegion(admin, server, region.getRegionName(),
-      versionOfClosingNode, dest, transitionInZK);
+      dest);
   }
 
   public boolean sendRegionClose(ServerName server,
-      HRegionInfo region, int versionOfClosingNode) throws IOException {
-    return sendRegionClose(server, region, versionOfClosingNode, null, true);
+      HRegionInfo region) throws IOException {
+    return sendRegionClose(server, region, null);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java
deleted file mode 100644
index b01434e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- *
- * 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.handler;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-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.master.RegionState;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-
-/**
- * Handles CLOSED region event on Master.
- * <p>
- * If table is being disabled, deletes ZK unassigned node and removes from
- * regions in transition.
- * <p>
- * Otherwise, assigns the region to another server.
- */
-@InterfaceAudience.Private
-public class ClosedRegionHandler extends EventHandler implements TotesHRegionInfo {
-  private static final Log LOG = LogFactory.getLog(ClosedRegionHandler.class);
-  private final AssignmentManager assignmentManager;
-  private final HRegionInfo regionInfo;
-  private final ClosedPriority priority;
-
-  private enum ClosedPriority {
-    META (1),
-    USER (2);
-
-    private final int value;
-    ClosedPriority(int value) {
-      this.value = value;
-    }
-    public int getValue() {
-      return value;
-    }
-  };
-
-  public ClosedRegionHandler(Server server, AssignmentManager assignmentManager,
-      HRegionInfo regionInfo) {
-    super(server, EventType.RS_ZK_REGION_CLOSED);
-    this.assignmentManager = assignmentManager;
-    this.regionInfo = regionInfo;
-    if(regionInfo.isMetaRegion()) {
-      priority = ClosedPriority.META;
-    } else {
-      priority = ClosedPriority.USER;
-    }
-  }
-
-  @Override
-  public int getPriority() {
-    return priority.getValue();
-  }
-
-  @Override
-  public HRegionInfo getHRegionInfo() {
-    return this.regionInfo;
-  }
-
-  @Override
-  public String toString() {
-    String name = "UnknownServerName";
-    if(server != null && server.getServerName() != null) {
-      name = server.getServerName().toString();
-    }
-    return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
-  }
-
-  @Override
-  public void process() {
-    LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
-    // Check if this table is being disabled or not
-    if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(),
-        ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
-        assignmentManager.getReplicasToClose().contains(regionInfo)) {
-      assignmentManager.offlineDisabledRegion(regionInfo);
-      return;
-    }
-    // ZK Node is in CLOSED state, assign it.
-    assignmentManager.getRegionStates().updateRegionState(
-      regionInfo, RegionState.State.CLOSED);
-    // This below has to do w/ online enable/disable of a table
-    assignmentManager.removeClosedRegion(regionInfo);
-    assignmentManager.assign(regionInfo, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
deleted file mode 100644
index a2dc41b..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- *
- * 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.handler;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-
-/**
- * Handles OPENED region event on Master.
- */
-@InterfaceAudience.Private
-public class OpenedRegionHandler extends EventHandler implements TotesHRegionInfo {
-  private static final Log LOG = LogFactory.getLog(OpenedRegionHandler.class);
-  private final AssignmentManager assignmentManager;
-  private final HRegionInfo regionInfo;
-  private final OpenedPriority priority;
-
-  private OpenRegionCoordination coordination;
-  private OpenRegionCoordination.OpenRegionDetails ord;
-
-  private enum OpenedPriority {
-    META (1),
-    SYSTEM (2),
-    USER (3);
-
-    private final int value;
-    OpenedPriority(int value) {
-      this.value = value;
-    }
-    public int getValue() {
-      return value;
-    }
-  };
-
-  public OpenedRegionHandler(Server server,
-      AssignmentManager assignmentManager, HRegionInfo regionInfo,
-      OpenRegionCoordination coordination,
-      OpenRegionCoordination.OpenRegionDetails ord) {
-    super(server, EventType.RS_ZK_REGION_OPENED);
-    this.assignmentManager = assignmentManager;
-    this.regionInfo = regionInfo;
-    this.coordination = coordination;
-    this.ord = ord;
-    if(regionInfo.isMetaRegion()) {
-      priority = OpenedPriority.META;
-    } else if(regionInfo.getTable()
-        .getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
-      priority = OpenedPriority.SYSTEM;
-    } else {
-      priority = OpenedPriority.USER;
-    }
-  }
-
-  @Override
-  public int getPriority() {
-    return priority.getValue();
-  }
-
-  @Override
-  public HRegionInfo getHRegionInfo() {
-    return this.regionInfo;
-  }
-
-  @Override
-  public String toString() {
-    String name = "UnknownServerName";
-    if(server != null && server.getServerName() != null) {
-      name = server.getServerName().toString();
-    }
-    return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
-  }
-
-  @Override
-  public void process() {
-    if (!coordination.commitOpenOnMasterSide(assignmentManager,regionInfo, ord)) {
-        assignmentManager.unassign(regionInfo);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 f6d798a..5d26ac8 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
@@ -32,7 +32,6 @@ 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager;
@@ -45,9 +44,6 @@ import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
-import org.apache.hadoop.hbase.util.ConfigUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Process server shutdown.
@@ -162,24 +158,15 @@ public class ServerShutdownHandler extends EventHandler {
           server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
           // Skip getting user regions if the server is stopped.
           if (!this.server.isStopped()) {
-            if (ConfigUtil.useZKForAssignment(server.getConfiguration())) {
-              hris = MetaTableAccessor.getServerUserRegions(this.server.getShortCircuitConnection(),
-                this.serverName).keySet();
-            } else {
-              // Not using ZK for assignment, regionStates has everything we want
-              hris = am.getRegionStates().getServerRegions(serverName);
-              if (hris != null) {
-                hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
-              }
+            hris = am.getRegionStates().getServerRegions(serverName);
+            if (hris != null) {
+              hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
             }
           }
           break;
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
-        } catch (IOException ioe) {
-          LOG.info("Received exception accessing hbase:meta during server shutdown of " +
-            serverName + ", retrying hbase:meta read", ioe);
         }
       }
       if (this.server.isStopped()) {
@@ -249,15 +236,8 @@ public class ServerShutdownHandler extends EventHandler {
                   LOG.info("Skip assigning region in transition on other server" + rit);
                   continue;
                 }
-                try{
-                  //clean zk node
-                  LOG.info("Reassigning region with rs = " + rit + " and deleting zk node if exists");
-                  ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), hri);
-                  regionStates.updateRegionState(hri, State.OFFLINE);
-                } catch (KeeperException ke) {
-                  this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
-                  return;
-                }
+                LOG.info("Reassigning region with rs = " + rit);
+                regionStates.updateRegionState(hri, State.OFFLINE);
               } else if (regionStates.isRegionInState(
                   hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
                 regionStates.updateRegionState(hri, State.OFFLINE);
@@ -274,7 +254,6 @@ public class ServerShutdownHandler extends EventHandler {
                 // but though we did assign we will not be clearing the znode in CLOSING state.
                 // Doing this will have no harm. See HBASE-5927
                 regionStates.updateRegionState(hri, State.OFFLINE);
-                am.deleteClosingOrClosedNode(hri, rit.getServerName());
                 am.offlineDisabledRegion(hri);
               } else {
                 LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/UpgradeTo96.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/UpgradeTo96.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/UpgradeTo96.java
deleted file mode 100644
index 6df2eab..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/UpgradeTo96.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/**
- * 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.migration;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileV1Detector;
-import org.apache.hadoop.hbase.util.ZKDataMigrator;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class UpgradeTo96 extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(UpgradeTo96.class);
-
-  private Options options = new Options();
-  /**
-   * whether to do overall upgrade (namespace and znodes)
-   */
-  private boolean upgrade;
-  /**
-   * whether to check for HFileV1
-   */
-  private boolean checkForHFileV1;
-  /**
-   * Path of directory to check for HFileV1
-   */
-  private String dirToCheckForHFileV1;
-
-  UpgradeTo96() {
-    setOptions();
-  }
-
-  private void setOptions() {
-    options.addOption("h", "help", false, "Help");
-    options.addOption(new Option("check", false, "Run upgrade check; looks for HFileV1 "
-        + " under ${hbase.rootdir} or provided 'dir' directory."));
-    options.addOption(new Option("execute", false, "Run upgrade; zk and hdfs must be up, hbase down"));
-    Option pathOption = new Option("dir", true,
-        "Relative path of dir to check for HFileV1s.");
-    pathOption.setRequired(false);
-    options.addOption(pathOption);
-  }
-
-  private boolean parseOption(String[] args) throws ParseException {
-    if (args.length == 0) return false; // no args shows help.
-
-    CommandLineParser parser = new GnuParser();
-    CommandLine cmd = parser.parse(options, args);
-    if (cmd.hasOption("h")) {
-      return false;
-    }
-    if (cmd.hasOption("execute")) upgrade = true;
-    if (cmd.hasOption("check")) checkForHFileV1 = true;
-    if (checkForHFileV1 && cmd.hasOption("dir")) {
-      this.dirToCheckForHFileV1 = cmd.getOptionValue("dir");
-    }
-    return true;
-  }
-
-  private void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("$bin/hbase upgrade -check [-dir DIR]|-execute", options);
-    System.out.println("Read http://hbase.apache.org/book.html#upgrade0.96 before attempting upgrade");
-    System.out.println();
-    System.out.println("Example usage:");
-    System.out.println();
-    System.out.println("Run upgrade check; looks for HFileV1s under ${hbase.rootdir}:");
-    System.out.println(" $ bin/hbase upgrade -check");
-    System.out.println();
-    System.out.println("Run the upgrade: ");
-    System.out.println(" $ bin/hbase upgrade -execute");
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    if (!parseOption(args)) {
-      printUsage();
-      return -1;
-    }
-    if (checkForHFileV1) {
-      int res = doHFileV1Check();
-      if (res == 0) LOG.info("No HFileV1 found.");
-      else {
-        LOG.warn("There are some HFileV1, or corrupt files (files with incorrect major version).");
-      }
-      return res;
-    }
-    // if the user wants to upgrade, check for any HBase live process.
-    // If yes, prompt the user to stop them
-    else if (upgrade) {
-      if (isAnyHBaseProcessAlive()) {
-        LOG.error("Some HBase processes are still alive, or znodes not expired yet. "
-            + "Please stop them before upgrade or try after some time.");
-        throw new IOException("Some HBase processes are still alive, or znodes not expired yet");
-      }
-      return executeUpgrade();
-    }
-    return -1;
-  }
-
-  private boolean isAnyHBaseProcessAlive() throws IOException {
-    ZooKeeperWatcher zkw = null;
-    try {
-      zkw = new ZooKeeperWatcher(getConf(), "Check Live Processes.", new Abortable() {
-        private boolean aborted = false;
-
-        @Override
-        public void abort(String why, Throwable e) {
-          LOG.warn("Got aborted with reason: " + why + ", and error: " + e);
-          this.aborted = true;
-        }
-
-        @Override
-        public boolean isAborted() {
-          return this.aborted;
-        }
-
-      });
-      boolean liveProcessesExists = false;
-      if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
-        return false;
-      }
-      if (ZKUtil.checkExists(zkw, zkw.backupMasterAddressesZNode) != -1) {
-        List<String> backupMasters = ZKUtil
-            .listChildrenNoWatch(zkw, zkw.backupMasterAddressesZNode);
-        if (!backupMasters.isEmpty()) {
-          LOG.warn("Backup master(s) " + backupMasters
-              + " are alive or backup-master znodes not expired.");
-          liveProcessesExists = true;
-        }
-      }
-      if (ZKUtil.checkExists(zkw, zkw.rsZNode) != -1) {
-        List<String> regionServers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
-        if (!regionServers.isEmpty()) {
-          LOG.warn("Region server(s) " + regionServers + " are alive or rs znodes not expired.");
-          liveProcessesExists = true;
-        }
-      }
-      if (ZKUtil.checkExists(zkw, zkw.getMasterAddressZNode()) != -1) {
-        byte[] data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
-        if (data != null && !Bytes.equals(data, HConstants.EMPTY_BYTE_ARRAY)) {
-          LOG.warn("Active master at address " + Bytes.toString(data)
-              + " is still alive or master znode not expired.");
-          liveProcessesExists = true;
-        }
-      }
-      return liveProcessesExists;
-    } catch (Exception e) {
-      LOG.error("Got exception while checking live hbase processes", e);
-      throw new IOException(e);
-    } finally {
-      if (zkw != null) {
-        zkw.close();
-      }
-    }
-  }
-
-  private int doHFileV1Check() throws Exception {
-    String[] args = null;
-    if (dirToCheckForHFileV1 != null) args = new String[] { "-p" + dirToCheckForHFileV1 };
-    return ToolRunner.run(getConf(), new HFileV1Detector(), args);
-  }
-
-  /**
-   * Executes the upgrade process. It involves:
-   * <ul>
-   * <li> Upgrading Namespace
-   * <li> Upgrading Znodes
-   * <li> Log splitting
-   * </ul>
-   * @throws Exception
-   */
-  private int executeUpgrade() throws Exception {
-    executeTool("Namespace upgrade", new NamespaceUpgrade(),
-      new String[] { "--upgrade" }, 0);
-    executeTool("Znode upgrade", new ZKDataMigrator(), null, 0);
-    doOfflineLogSplitting();
-    return 0;
-  }
-
-  private void executeTool(String toolMessage, Tool tool, String[] args, int expectedResult)
-      throws Exception {
-    LOG.info("Starting " + toolMessage);
-    int res = ToolRunner.run(getConf(), tool, new String[] { "--upgrade" });
-    if (res != expectedResult) {
-      LOG.error(toolMessage + "returned " + res + ", expected " + expectedResult);
-      throw new Exception("Unexpected return code from " + toolMessage);
-    }
-    LOG.info("Successfully completed " + toolMessage);
-  }
-
-  /**
-   * Performs log splitting for all regionserver directories.
-   * @throws Exception
-   */
-  private void doOfflineLogSplitting() throws Exception {
-    LOG.info("Starting Log splitting");
-    final Path rootDir = FSUtils.getRootDir(getConf());
-    final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
-    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
-    FileStatus[] regionServerLogDirs = FSUtils.listStatus(fs, logDir);
-    if (regionServerLogDirs == null || regionServerLogDirs.length == 0) {
-      LOG.info("No log directories to split, returning");
-      return;
-    }
-    try {
-      for (FileStatus regionServerLogDir : regionServerLogDirs) {
-        // split its log dir, if exists
-        HLogSplitter.split(rootDir, regionServerLogDir.getPath(), oldLogDir, fs, getConf());
-      }
-      LOG.info("Successfully completed Log splitting");
-    } catch (Exception e) {
-      LOG.error("Got exception while doing Log splitting ", e);
-      throw e;
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(HBaseConfiguration.create(), new UpgradeTo96(), args));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 9c28bfc..46a1e51 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
@@ -76,7 +76,6 @@ import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
@@ -123,7 +122,6 @@ import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.ByteStringer;
 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;
@@ -149,6 +147,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.ServiceException;
 
@@ -163,6 +162,12 @@ public class HRegionServer extends HasThread implements
 
   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
 
+  /**
+   * For testing only!  Set to true to skip notifying region assignment to master .
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
+  public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
+
   /*
    * Strings to be used in forming the exception message for
    * RegionsAlreadyInTransitionException.
@@ -410,8 +415,6 @@ public class HRegionServer extends HasThread implements
 
   protected BaseCoordinatedStateManager csm;
 
-  private final boolean useZKForAssignment;
-
   /**
    * Starts a HRegionServer at the default location.
    * @param conf
@@ -427,10 +430,9 @@ public class HRegionServer extends HasThread implements
    * @param conf
    * @param csm implementation of CoordinatedStateManager to be used
    * @throws IOException
-   * @throws InterruptedException
    */
   public HRegionServer(Configuration conf, CoordinatedStateManager csm)
-      throws IOException, InterruptedException {
+      throws IOException {
     this.fsOk = true;
     this.conf = conf;
     checkCodecs(this.conf);
@@ -479,8 +481,6 @@ public class HRegionServer extends HasThread implements
       }
     };
 
-    useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
-
     // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
     // underlying hadoop hdfs accessors will be going against wrong filesystem
     // (unless all is set to defaults).
@@ -1719,14 +1719,12 @@ public class HRegionServer extends HasThread implements
     // Update flushed sequence id of a recovering region in ZK
     updateRecoveringRegionLastFlushedSequenceId(r);
 
-    // Update ZK, or META
     if (r.getRegionInfo().isMetaRegion()) {
       MetaTableLocator.setMetaLocation(getZooKeeper(), serverName);
-    } else if (useZKForAssignment) {
-      MetaTableAccessor.updateRegionLocation(shortCircuitConnection, r.getRegionInfo(),
-        this.serverName, openSeqNum);
     }
-    if (!useZKForAssignment && !reportRegionStateTransition(
+
+    // Notify master
+    if (!reportRegionStateTransition(
         TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
       throw new IOException("Failed to report opened region to master: "
         + r.getRegionNameAsString());
@@ -1743,6 +1741,22 @@ public class HRegionServer extends HasThread implements
   @Override
   public boolean reportRegionStateTransition(
       TransitionCode code, long openSeqNum, HRegionInfo... hris) {
+    if (TEST_SKIP_REPORTING_TRANSITION) {
+      // This is for testing only in case there is no master
+      // to handle the region transition report at all.
+      if (code == TransitionCode.OPENED) {
+        Preconditions.checkArgument(hris != null && hris.length == 1);
+        try {
+          MetaTableAccessor.updateRegionLocation(shortCircuitConnection,
+            hris[0], serverName, openSeqNum);
+          return true;
+        } catch (IOException e) {
+          LOG.info("Failed to update meta", e);
+          return false;
+        }
+      }
+    }
+
     ReportRegionStateTransitionRequest.Builder builder =
       ReportRegionStateTransitionRequest.newBuilder();
     builder.setServer(ProtobufUtil.toServerName(serverName));
@@ -2428,9 +2442,7 @@ public class HRegionServer extends HasThread implements
    */
   private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
     try {
-      CloseRegionCoordination.CloseRegionDetails details =
-        csm.getCloseRegionCoordination().getDetaultDetails();
-      if (!closeRegion(region.getEncodedName(), abort, details, null)) {
+      if (!closeRegion(region.getEncodedName(), abort, null)) {
         LOG.warn("Failed to close " + region.getRegionNameAsString() +
             " - ignoring and continuing");
       }
@@ -2455,13 +2467,11 @@ public class HRegionServer extends HasThread implements
    *
    * @param encodedName Region to close
    * @param abort True if we are aborting
-   * @param crd details about closing region coordination-coordinated task
    * @return True if closed a region.
    * @throws NotServingRegionException if the region is not online
    * @throws RegionAlreadyInTransitionException if the region is already closing
    */
-  protected boolean closeRegion(String encodedName, final boolean abort,
-      CloseRegionCoordination.CloseRegionDetails crd, final ServerName sn)
+  protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
       throws NotServingRegionException, RegionAlreadyInTransitionException {
     //Check for permissions to close.
     HRegion actualRegion = this.getFromOnlineRegions(encodedName);
@@ -2485,7 +2495,7 @@ public class HRegionServer extends HasThread implements
         // We're going to try to do a standard close then.
         LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
             " Doing a standard close now");
-        return closeRegion(encodedName, abort, crd, sn);
+        return closeRegion(encodedName, abort, sn);
       }
       // Let's get the region from the online region list again
       actualRegion = this.getFromOnlineRegions(encodedName);
@@ -2519,11 +2529,9 @@ public class HRegionServer extends HasThread implements
     CloseRegionHandler crh;
     final HRegionInfo hri = actualRegion.getRegionInfo();
     if (hri.isMetaRegion()) {
-      crh = new CloseMetaHandler(this, this, hri, abort,
-        csm.getCloseRegionCoordination(), crd);
+      crh = new CloseMetaHandler(this, this, hri, abort);
     } else {
-      crh = new CloseRegionHandler(this, this, hri, abort,
-        csm.getCloseRegionCoordination(), crd, sn);
+      crh = new CloseRegionHandler(this, this, hri, abort, sn);
     }
     this.service.submit(crh);
     return true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 04e7995..10da06d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -50,11 +50,11 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.Delete;
@@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.OperationConflictException;
 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
@@ -146,7 +145,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 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;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -927,10 +925,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
       requestCount.increment();
       LOG.info("Close " + encodedRegionName + ", moving to " + sn);
-      CloseRegionCoordination.CloseRegionDetails crd = regionServer.getCoordinatedStateManager()
-        .getCloseRegionCoordination().parseFromProtoRequest(request);
-
-      boolean closed = regionServer.closeRegion(encodedRegionName, false, crd, sn);
+      boolean closed = regionServer.closeRegion(encodedRegionName, false, sn);
       CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);
       return builder.build();
     } catch (IOException ie) {
@@ -1236,11 +1231,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
     for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
       final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
-      OpenRegionCoordination coordination = regionServer.getCoordinatedStateManager().
-        getOpenRegionCoordination();
-      OpenRegionCoordination.OpenRegionDetails ord =
-        coordination.parseFromProtoRequest(regionOpenInfo);
-
       HTableDescriptor htd;
       try {
         final HRegion onlineRegion = regionServer.getFromOnlineRegions(region.getEncodedName());
@@ -1284,10 +1274,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           region.getEncodedNameAsBytes(), Boolean.TRUE);
 
         if (Boolean.FALSE.equals(previous)) {
-          // There is a close in progress. We need to mark this open as failed in ZK.
-
-          coordination.tryTransitionFromOfflineToFailedOpen(regionServer, region, ord);
-
+          // There is a close in progress. This should not happen any more.
           throw new RegionAlreadyInTransitionException("Received OPEN for the region:"
             + region.getRegionNameAsString() + " , which we are already trying to CLOSE ");
         }
@@ -1324,12 +1311,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           // Need to pass the expected version in the constructor.
           if (region.isMetaRegion()) {
             regionServer.service.submit(new OpenMetaHandler(
-              regionServer, regionServer, region, htd, coordination, ord));
+              regionServer, regionServer, region, htd));
           } else {
             regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(),
               regionOpenInfo.getFavoredNodesList());
             regionServer.service.submit(new OpenRegionHandler(
-              regionServer, regionServer, region, htd, coordination, ord));
+              regionServer, regionServer, region, htd));
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 2db8d7e..cb28c9a 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
@@ -31,22 +31,17 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.RegionMergeCoordination.RegionMergeDetails;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.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.zookeeper.KeeperException;
 
 /**
  * Executes region merge as a "transaction". It is similar with
@@ -89,7 +84,6 @@ public class RegionMergeTransaction {
   private final Path mergesdir;
   // We only merge adjacent regions if forcible is false
   private final boolean forcible;
-  private boolean useCoordinationForAssignment;
 
   /**
    * Types to add to the transaction journal. Each enum is a step in the merge
@@ -141,8 +135,6 @@ public class RegionMergeTransaction {
 
   private RegionServerCoprocessorHost rsCoprocessorHost = null;
 
-  private RegionMergeDetails rmd;
-
   /**
    * Constructor
    * @param a region a to merge
@@ -231,14 +223,6 @@ public class RegionMergeTransaction {
    */
   public HRegion execute(final Server server,
  final RegionServerServices services) throws IOException {
-    useCoordinationForAssignment =
-        server == null ? true : ConfigUtil.useZKForAssignment(server.getConfiguration());
-    if (rmd == null) {
-      rmd =
-          server != null && server.getCoordinatedStateManager() != null ? ((BaseCoordinatedStateManager) server
-              .getCoordinatedStateManager()).getRegionMergeCoordination().getDefaultDetails()
-              : null;
-    }
     if (rsCoprocessorHost == null) {
       rsCoprocessorHost = server != null ?
         ((HRegionServer) server).getRegionServerCoprocessorHost() : null;
@@ -253,11 +237,6 @@ public class RegionMergeTransaction {
   public HRegion stepsAfterPONR(final Server server, final RegionServerServices services,
       HRegion mergedRegion) throws IOException {
     openMergedRegion(server, services, mergedRegion);
-    if (useCoordination(server)) {
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-          .getRegionMergeCoordination().completeRegionMergeTransaction(services, mergedRegionInfo,
-            region_a, region_b, rmd, mergedRegion);
-    }
     if (rsCoprocessorHost != null) {
       rsCoprocessorHost.postMerge(this.region_a, this.region_b, mergedRegion);
     }
@@ -322,35 +301,16 @@ 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 && useCoordinationForAssignment) {
-      if (metaEntries.isEmpty()) {
-        MetaTableAccessor.mergeRegions(server.getShortCircuitConnection(),
-          mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
-          server.getServerName());
-      } else {
-        mergeRegionsAndPutMetaEntries(server.getShortCircuitConnection(),
-          mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
-          server.getServerName(), metaEntries);
-      }
-    } else if (services != null && !useCoordinationForAssignment) {
-      if (!services.reportRegionStateTransition(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());
-      }
+    if (services != null && !services.reportRegionStateTransition(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;
   }
 
-  private void mergeRegionsAndPutMetaEntries(HConnection hConnection,
-      HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB,
-      ServerName serverName, List<Mutation> metaEntries) throws IOException {
-    prepareMutationsForMerge(mergedRegion, regionA, regionB, serverName, metaEntries);
-    MetaTableAccessor.mutateMetaTable(hConnection, metaEntries);
-  }
-
   public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
       HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
     HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
@@ -380,40 +340,13 @@ public class RegionMergeTransaction {
 
   public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
       boolean testing) throws IOException {
-    if (rmd == null) {
-      rmd =
-          server != null && server.getCoordinatedStateManager() != null ? ((BaseCoordinatedStateManager) server
-              .getCoordinatedStateManager()).getRegionMergeCoordination().getDefaultDetails()
-              : null;
-    }
-
-    // If server doesn't have a coordination state manager, don't do coordination actions.
-    if (useCoordination(server)) {
-      try {
-        ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-            .getRegionMergeCoordination().startRegionMergeTransaction(mergedRegionInfo,
-              server.getServerName(), region_a.getRegionInfo(), region_b.getRegionInfo());
-      } catch (IOException e) {
-        throw new IOException("Failed to start region merge transaction for "
-            + this.mergedRegionInfo.getRegionNameAsString(), e);
-      }
-    } else if (services != null && !useCoordinationForAssignment) {
-      if (!services.reportRegionStateTransition(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());
-      }
+    if (services != null && !services.reportRegionStateTransition(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);
-    if (useCoordination(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.
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-          .getRegionMergeCoordination().waitForRegionMergeTransaction(services, mergedRegionInfo,
-            region_a, region_b, rmd);
-    }
 
     this.region_a.getRegionFileSystem().createMergesDir();
     this.journal.add(JournalEntry.CREATED_MERGE_DIR);
@@ -432,19 +365,6 @@ public class RegionMergeTransaction {
     // clean this up.
     mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
 
-    if (useCoordination(server)) {
-      try {
-        // Do the final check in case any merging region is moved somehow. If so, the transition
-        // will fail.
-        ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-            .getRegionMergeCoordination().confirmRegionMergeTransaction(this.mergedRegionInfo,
-              region_a.getRegionInfo(), region_b.getRegionInfo(), server.getServerName(), rmd);
-      } catch (IOException e) {
-        throw new IOException("Failed setting MERGING on "
-            + this.mergedRegionInfo.getRegionNameAsString(), e);
-      }
-    }
-
     // Log to the journal that we are creating merged region. We could fail
     // halfway through. If we do, we could have left
     // stuff in fs that needs cleanup -- a storefile or two. Thats why we
@@ -578,20 +498,13 @@ public class RegionMergeTransaction {
     merged.openHRegion(reporter);
 
     if (services != null) {
-      try {
-        if (useCoordinationForAssignment) {
-          services.postOpenDeployTasks(merged);
-        } else if (!services.reportRegionStateTransition(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);
+      if (!services.reportRegionStateTransition(TransitionCode.MERGED,
+          mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
+        throw new IOException("Failed to report merged region to master: "
+          + mergedRegionInfo.getShortNameToLog());
       }
+      services.addToOnlineRegions(merged);
     }
-
   }
 
   /**
@@ -652,10 +565,7 @@ public class RegionMergeTransaction {
       switch (je) {
 
         case SET_MERGING:
-        if (useCoordination(server)) {
-          ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-              .getRegionMergeCoordination().clean(this.mergedRegionInfo);
-          } else if (services != null && !useCoordinationForAssignment
+          if (services != null
               && !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
                   mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
             return false;
@@ -734,13 +644,6 @@ public class RegionMergeTransaction {
     return this.mergesdir;
   }
 
-  private boolean useCoordination(final Server server) {
-    return server != null && useCoordinationForAssignment
-        && server.getCoordinatedStateManager() != null;
-  }
-
-
-
   /**
    * Checks if the given region has merge qualifier in hbase:meta
    * @param services

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 c5b29e6..30b55dd 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
@@ -39,16 +39,11 @@ 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.MetaTableAccessor;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.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;
@@ -90,8 +85,6 @@ public class SplitTransaction {
   private HRegionInfo hri_a;
   private HRegionInfo hri_b;
   private long fileSplitTimeout = 30000;
-  public SplitTransactionCoordination.SplitTransactionDetails std;
-  boolean useZKForAssignment;
 
   /*
    * Row to split around
@@ -275,52 +268,23 @@ 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 && useZKForAssignment) {
-      if (metaEntries == null || metaEntries.isEmpty()) {
-        MetaTableAccessor.splitRegion(server.getShortCircuitConnection(),
-          parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(),
-          daughterRegions.getSecond().getRegionInfo(), server.getServerName());
-      } else {
-        offlineParentInMetaAndputMetaEntries(server.getShortCircuitConnection(),
-          parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
-              .getSecond().getRegionInfo(), server.getServerName(), metaEntries);
-      }
-    } else if (services != null && !useZKForAssignment) {
-      if (!services.reportRegionStateTransition(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());
-      }
+    if (services != null && !services.reportRegionStateTransition(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;
   }
 
   public PairOfSameType<HRegion> stepsBeforePONR(final Server server,
       final RegionServerServices services, boolean testing) throws IOException {
-
-    if (useCoordinatedStateManager(server)) {
-      if (std == null) {
-        std =
-            ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-                .getSplitTransactionCoordination().getDefaultDetails();
-      }
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-          .getSplitTransactionCoordination().startSplitTransaction(parent, server.getServerName(),
-            hri_a, hri_b);
-    } else if (services != null && !useZKForAssignment) {
-      if (!services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
-          parent.getRegionInfo(), hri_a, hri_b)) {
-        throw new IOException("Failed to get ok from master to split "
-          + parent.getRegionNameAsString());
-      }
+    if (services != null && !services.reportRegionStateTransition(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);
-    if (useCoordinatedStateManager(server)) {
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-          .getSplitTransactionCoordination().waitForSplitTransaction(services, parent, hri_a,
-            hri_b, std);
-    }
 
     this.parent.getRegionFileSystem().createSplitsDir();
     this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
@@ -415,24 +379,14 @@ public class SplitTransaction {
           bOpener.getName(), bOpener.getException());
       }
       if (services != null) {
-        try {
-          if (useZKForAssignment) {
-            // add 2nd daughter first (see HBASE-4335)
-            services.postOpenDeployTasks(b);
-          } else if (!services.reportRegionStateTransition(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);
-          if (useZKForAssignment) {
-            services.postOpenDeployTasks(a);
-          }
-          services.addToOnlineRegions(a);
-        } catch (KeeperException ke) {
-          throw new IOException(ke);
+        if (!services.reportRegionStateTransition(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.addToOnlineRegions(a);
       }
     }
   }
@@ -450,13 +404,6 @@ public class SplitTransaction {
   public PairOfSameType<HRegion> execute(final Server server,
       final RegionServerServices services)
   throws IOException {
-    useZKForAssignment = server == null ? true :
-      ConfigUtil.useZKForAssignment(server.getConfiguration());
-    if (useCoordinatedStateManager(server)) {
-      std =
-          ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-              .getSplitTransactionCoordination().getDefaultDetails();
-    }
     PairOfSameType<HRegion> regions = createDaughters(server, services);
     if (this.parent.getCoprocessorHost() != null) {
       this.parent.getCoprocessorHost().preSplitAfterPONR();
@@ -468,44 +415,13 @@ public class SplitTransaction {
       final RegionServerServices services, PairOfSameType<HRegion> regions)
       throws IOException {
     openDaughters(server, services, regions.getFirst(), regions.getSecond());
-    if (useCoordinatedStateManager(server)) {
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-          .getSplitTransactionCoordination().completeSplitTransaction(services, regions.getFirst(),
-            regions.getSecond(), std, parent);
-    }
     // Coprocessor callback
     if (parent.getCoprocessorHost() != null) {
       parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
     }
-
-
     return regions;
   }
 
-  private void offlineParentInMetaAndputMetaEntries(HConnection hConnection,
-      HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
-      ServerName serverName, List<Mutation> metaEntries) throws IOException {
-    List<Mutation> mutations = metaEntries;
-    HRegionInfo copyOfParent = new HRegionInfo(parent);
-    copyOfParent.setOffline(true);
-    copyOfParent.setSplit(true);
-
-    //Put for parent
-    Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
-    MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
-    mutations.add(putParent);
-    
-    //Puts for daughters
-    Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);
-    Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
-
-    addLocation(putA, serverName, 1); //these are new regions, openSeqNum = 1 is fine.
-    addLocation(putB, serverName, 1);
-    mutations.add(putA);
-    mutations.add(putB);
-    MetaTableAccessor.mutateMetaTable(hConnection, mutations);
-  }
-
   public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
     p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
       Bytes.toBytes(sn.getHostAndPort()));
@@ -588,10 +504,6 @@ public class SplitTransaction {
     }
   }
 
-  private boolean useCoordinatedStateManager(final Server server) {
-    return server != null && useZKForAssignment && server.getCoordinatedStateManager() != null;
-  }
-
   private void splitStoreFiles(final Map<byte[], List<StoreFile>> hstoreFilesToSplit)
       throws IOException {
     if (hstoreFilesToSplit == null) {
@@ -707,10 +619,7 @@ public class SplitTransaction {
       switch(je) {
 
       case SET_SPLITTING:
-        if (useCoordinatedStateManager(server) && server instanceof HRegionServer) {
-          ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-              .getSplitTransactionCoordination().clean(this.parent.getRegionInfo());
-        } else if (services != null && !useZKForAssignment
+        if (services != null
             && !services.reportRegionStateTransition(TransitionCode.SPLIT_REVERTED,
                 parent.getRegionInfo(), hri_a, hri_b)) {
           return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
index df8ae23..14d9b17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
@@ -23,10 +23,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
 
 /**
- * Handles closing of the root region on a region server.
+ * Handles closing of the meta region on a region server.
  */
 @InterfaceAudience.Private
 public class CloseMetaHandler extends CloseRegionHandler {
@@ -35,9 +34,7 @@ public class CloseMetaHandler extends CloseRegionHandler {
   public CloseMetaHandler(final Server server,
       final RegionServerServices rsServices,
       final HRegionInfo regionInfo,
-      final boolean abort, CloseRegionCoordination closeRegionCoordination,
-      CloseRegionCoordination.CloseRegionDetails crd) {
-    super(server, rsServices, regionInfo, abort, closeRegionCoordination,
-      crd, EventType.M_RS_CLOSE_META);
+      final boolean abort) {
+    super(server, rsServices, regionInfo, abort, EventType.M_RS_CLOSE_META, null);
   }
 }


[02/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java
deleted file mode 100644
index fe992b6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/**
- * 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.migration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-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.MediumTests;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-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.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test upgrade from no namespace in 0.94 to namespace directory structure.
- * Mainly tests that tables are migrated and consistent. Also verifies
- * that snapshots have been migrated correctly.
- *
- * <p>Uses a tarball which is an image of an 0.94 hbase.rootdir.
- *
- * <p>Contains tables with currentKeys as the stored keys:
- * foo, ns1.foo, ns2.foo
- *
- * <p>Contains snapshots with snapshot{num}Keys as the contents:
- * snapshot1Keys, snapshot2Keys
- *
- * Image also contains _acl_ table with one region and two storefiles.
- * This is needed to test the acl table migration.
- *
- */
-@Category(MediumTests.class)
-public class TestNamespaceUpgrade {
-  static final Log LOG = LogFactory.getLog(TestNamespaceUpgrade.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final static String snapshot1Keys[] =
-      {"1","10","2","3","4","5","6","7","8","9"};
-  private final static String snapshot2Keys[] =
-      {"1","2","3","4","5","6","7","8","9"};
-  private final static String currentKeys[] =
-      {"1","2","3","4","5","6","7","8","9","A"};
-  private final static TableName tables[] =
-    {TableName.valueOf("foo"), TableName.valueOf("ns1.foo"), TableName.valueOf("ns.two.foo")};
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Start up our mini cluster on top of an 0.94 root.dir that has data from
-    // a 0.94 hbase run and see if we can migrate to 0.96
-    TEST_UTIL.startMiniZKCluster();
-    TEST_UTIL.startMiniDFSCluster(1);
-    Path testdir = TEST_UTIL.getDataTestDir("TestNamespaceUpgrade");
-    // Untar our test dir.
-    File untar = untar(new File(testdir.toString()));
-    // Now copy the untar up into hdfs so when we start hbase, we'll run from it.
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setBoolean("hbase.assignment.usezk", true);
-    FsShell shell = new FsShell(conf);
-    FileSystem fs = FileSystem.get(conf);
-    // find where hbase will root itself, so we can copy filesystem there
-    Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
-    if (!fs.isDirectory(hbaseRootDir.getParent())) {
-      // mkdir at first
-      fs.mkdirs(hbaseRootDir.getParent());
-    }
-    if(org.apache.hadoop.util.VersionInfo.getVersion().startsWith("2.")) {
-      LOG.info("Hadoop version is 2.x, pre-migrating snapshot dir");
-      FileSystem localFS = FileSystem.getLocal(conf);
-      if(!localFS.rename(new Path(untar.toString(), HConstants.OLD_SNAPSHOT_DIR_NAME),
-          new Path(untar.toString(), HConstants.SNAPSHOT_DIR_NAME))) {
-        throw new IllegalStateException("Failed to move snapshot dir to 2.x expectation");
-      }
-    }
-    doFsCommand(shell,
-      new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
-    doFsCommand(shell, new String [] {"-lsr", "/"});
-    // See whats in minihdfs.
-    Configuration toolConf = TEST_UTIL.getConfiguration();
-    conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
-    ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
-    assertTrue(FSUtils.getVersion(fs, hbaseRootDir).equals(HConstants.FILE_SYSTEM_VERSION));
-    doFsCommand(shell, new String [] {"-lsr", "/"});
-    TEST_UTIL.startMiniHBaseCluster(1, 1);
-
-    for(TableName table: tables) {
-      int count = 0;
-      for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new Scan())) {
-        assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
-      }
-      Assert.assertEquals(currentKeys.length, count);
-    }
-    assertEquals(2, TEST_UTIL.getHBaseAdmin().listNamespaceDescriptors().length);
-
-    //verify ACL table is migrated
-    HTable secureTable = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-    ResultScanner scanner = secureTable.getScanner(new Scan());
-    int count = 0;
-    for(Result r : scanner) {
-      count++;
-    }
-    assertEquals(3, count);
-    assertFalse(TEST_UTIL.getHBaseAdmin().tableExists(TableName.valueOf("_acl_")));
-
-    //verify ACL table was compacted
-    List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(secureTable.getName());
-    for(HRegion region : regions) {
-      assertEquals(1, region.getStores().size());
-    }
-  }
-
-   static File untar(final File testdir) throws IOException {
-    // Find the src data under src/test/data
-    final String datafile = "TestNamespaceUpgrade";
-    File srcTarFile = new File(
-      System.getProperty("project.build.testSourceDirectory", "src/test") +
-      File.separator + "data" + File.separator + datafile + ".tgz");
-    File homedir = new File(testdir.toString());
-    File tgtUntarDir = new File(homedir, "hbase");
-    if (tgtUntarDir.exists()) {
-      if (!FileUtil.fullyDelete(tgtUntarDir)) {
-        throw new IOException("Failed delete of " + tgtUntarDir.toString());
-      }
-    }
-    if (!srcTarFile.exists()) {
-      throw new IOException(srcTarFile+" does not exist");
-    }
-    LOG.info("Untarring " + srcTarFile + " into " + homedir.toString());
-    FileUtil.unTar(srcTarFile, homedir);
-    Assert.assertTrue(tgtUntarDir.exists());
-    return tgtUntarDir;
-  }
-
-  private static void doFsCommand(final FsShell shell, final String [] args)
-  throws Exception {
-    // Run the 'put' command.
-    int errcode = shell.run(args);
-    if (errcode != 0) throw new IOException("Failed put; errcode=" + errcode);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test (timeout=300000)
-  public void testSnapshots() throws IOException, InterruptedException {
-    String snapshots[][] = {snapshot1Keys, snapshot2Keys};
-    for(int i = 1; i <= snapshots.length; i++) {
-      for(TableName table: tables) {
-        TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot"+i, TableName.valueOf(table+"_clone"+i));
-        FSUtils.logFileSystemState(FileSystem.get(TEST_UTIL.getConfiguration()),
-            FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
-            LOG);
-        int count = 0;
-        for(Result res: new HTable(TEST_UTIL.getConfiguration(), table+"_clone"+i).getScanner(new
-            Scan())) {
-          assertEquals(snapshots[i-1][count++], Bytes.toString(res.getRow()));
-        }
-        Assert.assertEquals(table+"_snapshot"+i, snapshots[i-1].length, count);
-      }
-    }
-  }
-
-  @Test (timeout=300000)
-  public void testRenameUsingSnapshots() throws Exception {
-    String newNS = "newNS";
-    TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(newNS).build());
-    for(TableName table: tables) {
-      int count = 0;
-      for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new
-          Scan())) {
-        assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
-      }
-      TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot3", table);
-      final TableName newTableName =
-        TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
-      TEST_UTIL.getHBaseAdmin().cloneSnapshot(table + "_snapshot3", newTableName);
-      Thread.sleep(1000);
-      count = 0;
-      for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
-          Scan())) {
-        assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
-      }
-      FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath()
-          , LOG);
-      Assert.assertEquals(newTableName + "", currentKeys.length, count);
-      TEST_UTIL.getHBaseAdmin().flush(newTableName.toBytes());
-      TEST_UTIL.getHBaseAdmin().majorCompact(newTableName.toBytes());
-      TEST_UTIL.waitFor(30000, new Waiter.Predicate<IOException>() {
-        @Override
-        public boolean evaluate() throws IOException {
-          try {
-            return TEST_UTIL.getHBaseAdmin().getCompactionState(newTableName.toBytes()) ==
-                AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
-          } catch (InterruptedException e) {
-            throw new IOException(e);
-          }
-        }
-      });
-    }
-
-    String nextNS = "nextNS";
-    TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(nextNS).build());
-    for(TableName table: tables) {
-      TableName srcTable = TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
-      TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot4", srcTable);
-      TableName newTableName =
-        TableName.valueOf(nextNS + TableName.NAMESPACE_DELIM + table + "_clone4");
-      TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot4", newTableName);
-      FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(),
-        LOG);
-      int count = 0;
-      for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
-          Scan())) {
-        assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
-      }
-      Assert.assertEquals(newTableName + "", currentKeys.length, count);
-    }
-  }
-
-  @Test (timeout=300000)
-  public void testOldDirsAreGonePostMigration() throws IOException {
-    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
-    List <String> dirs = new ArrayList<String>(NamespaceUpgrade.NON_USER_TABLE_DIRS);
-    // Remove those that are not renamed
-    dirs.remove(HConstants.HBCK_SIDELINEDIR_NAME);
-    dirs.remove(HConstants.SNAPSHOT_DIR_NAME);
-    dirs.remove(HConstants.HBASE_TEMP_DIRECTORY);
-    for (String dir: dirs) {
-      assertFalse(fs.exists(new Path(hbaseRootDir, dir)));
-    }
-  }
-
-  @Test (timeout=300000)
-  public void testNewDirsArePresentPostMigration() throws IOException {
-    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    // Below list does not include 'corrupt' because there is no 'corrupt' in the tgz
-    String [] newdirs = new String [] {HConstants.BASE_NAMESPACE_DIR,
-      HConstants.HREGION_LOGDIR_NAME};
-    Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
-    for (String dir: newdirs) {
-      assertTrue(dir, fs.exists(new Path(hbaseRootDir, dir)));
-    }
-  }
-
-  @Test (timeout = 300000)
-  public void testACLTableMigration() throws IOException {
-    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("testACLTable");
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-    Configuration conf = TEST_UTIL.getConfiguration();
-    byte[] FAMILY = Bytes.toBytes("l");
-    byte[] QUALIFIER = Bytes.toBytes("testUser");
-    byte[] VALUE = Bytes.toBytes("RWCA");
-
-    // Create a Region
-    HTableDescriptor aclTable = new HTableDescriptor(TableName.valueOf("testACLTable"));
-    aclTable.addFamily(new HColumnDescriptor(FAMILY));
-    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootDir);
-    fstd.createTableDescriptor(aclTable);
-    HRegionInfo hriAcl = new HRegionInfo(aclTable.getTableName(), null, null);
-    HRegion region = HRegion.createHRegion(hriAcl, rootDir, conf, aclTable);
-    try {
-      // Create rows
-      Put p = new Put(Bytes.toBytes("-ROOT-"));
-      p.addImmutable(FAMILY, QUALIFIER, VALUE);
-      region.put(p);
-      p = new Put(Bytes.toBytes(".META."));
-      p.addImmutable(FAMILY, QUALIFIER, VALUE);
-      region.put(p);
-      p = new Put(Bytes.toBytes("_acl_"));
-      p.addImmutable(FAMILY, QUALIFIER, VALUE);
-      region.put(p);
-
-      NamespaceUpgrade upgrade = new NamespaceUpgrade();
-      upgrade.updateAcls(region);
-
-      // verify rows -ROOT- is removed
-      Get g = new Get(Bytes.toBytes("-ROOT-"));
-      Result r = region.get(g);
-      assertTrue(r == null || r.size() == 0);
-
-      // verify rows _acl_ is renamed to hbase:acl
-      g = new Get(AccessControlLists.ACL_TABLE_NAME.toBytes());
-      r = region.get(g);
-      assertTrue(r != null && r.size() == 1);
-      assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
-
-      // verify rows .META. is renamed to hbase:meta
-      g = new Get(TableName.META_TABLE_NAME.toBytes());
-      r = region.get(g);
-      assertTrue(r != null && r.size() == 1);
-      assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
-    } finally {
-      region.close();
-      // Delete the region
-      HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
-        FSUtils.getTableDir(rootDir, hriAcl.getTable()), hriAcl);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java
deleted file mode 100644
index ab293f2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/**
- * 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.migration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.io.FileLink;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ReplicationPeer;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileV1Detector;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Upgrade to 0.96 involves detecting HFileV1 in existing cluster, updating namespace and
- * updating znodes. This class tests for HFileV1 detection and upgrading znodes.
- * Uprading namespace is tested in {@link TestNamespaceUpgrade}.
- */
-@Category(MediumTests.class)
-public class TestUpgradeTo96 {
-
-  static final Log LOG = LogFactory.getLog(TestUpgradeTo96.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  /**
-   * underlying file system instance
-   */
-  private static FileSystem fs;
-  /**
-   * hbase root dir
-   */
-  private static Path hbaseRootDir;
-  private static ZooKeeperWatcher zkw;
-  /**
-   * replication peer znode (/hbase/replication/peers)
-   */
-  private static String replicationPeerZnode;
-  /**
-   * znode of a table
-   */
-  private static String tableAZnode;
-  private static ReplicationPeer peer1;
-  /**
-   * znode for replication peer1 (/hbase/replication/peers/1)
-   */
-  private static String peer1Znode;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Start up the mini cluster on top of an 0.94 root.dir that has data from
-    // a 0.94 hbase run and see if we can migrate to 0.96
-    TEST_UTIL.startMiniZKCluster();
-    TEST_UTIL.startMiniDFSCluster(1);
-
-    hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
-    fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), hbaseRootDir);
-    zkw = TEST_UTIL.getZooKeeperWatcher();
-
-    Path testdir = TEST_UTIL.getDataTestDir("TestUpgradeTo96");
-    // get the untar 0.94 file structure
-
-    set94FSLayout(testdir);
-    setUp94Znodes();
-  }
-
-  /**
-   * Lays out 0.94 file system layout using {@link TestNamespaceUpgrade} apis.
-   * @param testdir
-   * @throws IOException
-   * @throws Exception
-   */
-  private static void set94FSLayout(Path testdir) throws IOException, Exception {
-    File untar = TestNamespaceUpgrade.untar(new File(testdir.toString()));
-    if (!fs.exists(hbaseRootDir.getParent())) {
-      // mkdir at first
-      fs.mkdirs(hbaseRootDir.getParent());
-    }
-    FsShell shell = new FsShell(TEST_UTIL.getConfiguration());
-    shell.run(new String[] { "-put", untar.toURI().toString(), hbaseRootDir.toString() });
-    // See whats in minihdfs.
-    shell.run(new String[] { "-lsr", "/" });
-  }
-
-  /**
-   * Sets znodes used in 0.94 version. Only table and replication znodes will be upgraded to PB,
-   * others would be deleted.
-   * @throws KeeperException
-   */
-  private static void setUp94Znodes() throws IOException, KeeperException {
-    // add some old znodes, which would be deleted after upgrade.
-    String rootRegionServerZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
-    ZKUtil.createWithParents(zkw, rootRegionServerZnode);
-    ZKUtil.createWithParents(zkw, zkw.backupMasterAddressesZNode);
-    // add table znode, data of its children would be protobuffized
-    tableAZnode = ZKUtil.joinZNode(zkw.tableZNode, "a");
-    ZKUtil.createWithParents(zkw, tableAZnode,
-      Bytes.toBytes(ZooKeeperProtos.Table.State.ENABLED.toString()));
-    // add replication znodes, data of its children would be protobuffized
-    String replicationZnode = ZKUtil.joinZNode(zkw.baseZNode, "replication");
-    replicationPeerZnode = ZKUtil.joinZNode(replicationZnode, "peers");
-    peer1Znode = ZKUtil.joinZNode(replicationPeerZnode, "1");
-    peer1 = ReplicationPeer.newBuilder().setClusterkey("abc:123:/hbase").build();
-    ZKUtil.createWithParents(zkw, peer1Znode, Bytes.toBytes(peer1.getClusterkey()));
-  }
-
-  /**
-   * Tests a 0.94 filesystem for any HFileV1.
-   * @throws Exception
-   */
-  @Test
-  public void testHFileV1Detector() throws Exception {
-    assertEquals(0, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
-  }
-
-  /**
-   * Creates a corrupt file, and run HFileV1 detector tool
-   * @throws Exception
-   */
-  @Test
-  public void testHFileV1DetectorWithCorruptFiles() throws Exception {
-    // add a corrupt file.
-    Path tablePath = new Path(hbaseRootDir, "foo");
-    FileStatus[] regionsDir = fs.listStatus(tablePath);
-    if (regionsDir == null) throw new IOException("No Regions found for table " + "foo");
-    Path columnFamilyDir = null;
-    Path targetRegion = null;
-    for (FileStatus s : regionsDir) {
-      if (fs.exists(new Path(s.getPath(), HRegionFileSystem.REGION_INFO_FILE))) {
-        targetRegion = s.getPath();
-        break;
-      }
-    }
-    FileStatus[] cfs = fs.listStatus(targetRegion);
-    for (FileStatus f : cfs) {
-      if (f.isDirectory()) {
-        columnFamilyDir = f.getPath();
-        break;
-      }
-    }
-    LOG.debug("target columnFamilyDir: " + columnFamilyDir);
-    // now insert a corrupt file in the columnfamily.
-    Path corruptFile = new Path(columnFamilyDir, "corrupt_file");
-    if (!fs.createNewFile(corruptFile)) throw new IOException("Couldn't create corrupt file: "
-        + corruptFile);
-    assertEquals(1, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
-    // remove the corrupt file
-    FileSystem.get(TEST_UTIL.getConfiguration()).delete(corruptFile, false);
-  }
-
-  @Test
-  public void testHFileLink() throws Exception {
-    // pass a link, and verify that correct paths are returned.
-    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
-    Path aFileLink = new Path(rootDir, "table/2086db948c48/cf/table=21212abcdc33-0906db948c48");
-    Path preNamespaceTablePath = new Path(rootDir, "table/21212abcdc33/cf/0906db948c48");
-    Path preNamespaceArchivePath =
-      new Path(rootDir, ".archive/table/21212abcdc33/cf/0906db948c48");
-    Path preNamespaceTempPath = new Path(rootDir, ".tmp/table/21212abcdc33/cf/0906db948c48");
-    boolean preNSTablePathExists = false;
-    boolean preNSArchivePathExists = false;
-    boolean preNSTempPathExists = false;
-    assertTrue(HFileLink.isHFileLink(aFileLink));
-    HFileLink hFileLink = new HFileLink(TEST_UTIL.getConfiguration(), aFileLink);
-    assertTrue(hFileLink.getArchivePath().toString().startsWith(rootDir.toString()));
-
-    HFileV1Detector t = new HFileV1Detector();
-    t.setConf(TEST_UTIL.getConfiguration());
-    FileLink fileLink = t.getFileLinkWithPreNSPath(aFileLink);
-    //assert it has 6 paths (2 NS, 2 Pre NS, and 2 .tmp)  to look.
-    assertTrue(fileLink.getLocations().length == 6);
-    for (Path p : fileLink.getLocations()) {
-      if (p.equals(preNamespaceArchivePath)) preNSArchivePathExists = true;
-      if (p.equals(preNamespaceTablePath)) preNSTablePathExists = true;
-      if (p.equals(preNamespaceTempPath)) preNSTempPathExists = true;
-    }
-    assertTrue(preNSArchivePathExists & preNSTablePathExists & preNSTempPathExists);
-  }
-
-  @Test
-  public void testADirForHFileV1() throws Exception {
-    Path tablePath = new Path(hbaseRootDir, "foo");
-    System.out.println("testADirForHFileV1: " + tablePath.makeQualified(fs));
-    System.out.println("Passed: " + hbaseRootDir + "/foo");
-    assertEquals(0,
-      ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), new String[] { "-p"
-          + "foo" }));
-  }
-
-  @Test
-  public void testZnodeMigration() throws Exception {
-    String rootRSZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
-    assertTrue(ZKUtil.checkExists(zkw, rootRSZnode) > -1);
-    ToolRunner.run(TEST_UTIL.getConfiguration(), new UpgradeTo96(), new String[] { "-execute" });
-    assertEquals(-1, ZKUtil.checkExists(zkw, rootRSZnode));
-    byte[] data = ZKUtil.getData(zkw, tableAZnode);
-    assertTrue(ProtobufUtil.isPBMagicPrefix(data));
-    checkTableState(data, ZooKeeperProtos.Table.State.ENABLED);
-    // ensure replication znodes are there, and protobuffed.
-    data = ZKUtil.getData(zkw, peer1Znode);
-    assertTrue(ProtobufUtil.isPBMagicPrefix(data));
-    checkReplicationPeerData(data, peer1);
-  }
-
-  private void checkTableState(byte[] data, State expectedState)
-      throws InvalidProtocolBufferException {
-    ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
-    int magicLen = ProtobufUtil.lengthOfPBMagic();
-    ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
-    assertTrue(t.getState() == expectedState);
-  }
-
-  private void checkReplicationPeerData(byte[] data, ReplicationPeer peer)
-      throws InvalidProtocolBufferException {
-    int magicLen = ProtobufUtil.lengthOfPBMagic();
-    ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
-    assertEquals(builder.mergeFrom(data, magicLen, data.length - magicLen).build().getClusterkey(),
-      peer.getClusterkey());
-
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniHBaseCluster();
-    TEST_UTIL.shutdownMiniDFSCluster();
-    TEST_UTIL.shutdownMiniZKCluster();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 8de605d..6cec1e2 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
@@ -34,16 +34,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Chore;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
@@ -52,14 +51,13 @@ import org.apache.hadoop.hbase.client.MetaScanner;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
 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;
@@ -74,6 +72,7 @@ import com.google.common.collect.Sets;
 import com.google.protobuf.ServiceException;
 
 @Category(LargeTests.class)
+@SuppressWarnings("deprecation")
 public class TestEndToEndSplitTransaction {
   private static final Log LOG = LogFactory.getLog(TestEndToEndSplitTransaction.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -109,7 +108,6 @@ 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
@@ -126,14 +124,10 @@ public class TestEndToEndSplitTransaction {
     // 3. finish phase II
     // note that this replicates some code from SplitTransaction
     // 2nd daughter first
-    if (split.useZKForAssignment) {
-      server.postOpenDeployTasks(regions.getSecond());
-    } else {
     server.reportRegionStateTransition(
       RegionServerStatusProtos.RegionStateTransition.TransitionCode.SPLIT,
       region.getRegionInfo(), regions.getFirst().getRegionInfo(),
       regions.getSecond().getRegionInfo());
-    }
 
     // Add to online regions
     server.addToOnlineRegions(regions.getSecond());
@@ -143,21 +137,11 @@ public class TestEndToEndSplitTransaction {
     // past splitkey is ok.
     assertTrue(test(con, tableName, lastRow, server));
 
-    // first daughter second
-    if (split.useZKForAssignment) {
-      server.postOpenDeployTasks(regions.getFirst());
-    }
     // Add to online regions
     server.addToOnlineRegions(regions.getFirst());
     assertTrue(test(con, tableName, firstRow, server));
     assertTrue(test(con, tableName, lastRow, server));
 
-    if (split.useZKForAssignment) {
-      // 4. phase III
-      ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
-        .getSplitTransactionCoordination().completeSplitTransaction(server, regions.getFirst(),
-          regions.getSecond(), split.std, region);
-    }
     assertTrue(test(con, tableName, firstRow, server));
     assertTrue(test(con, tableName, lastRow, server));
   }
@@ -341,7 +325,7 @@ public class TestEndToEndSplitTransaction {
         verifyStartEndKeys(keys);
 
         //HTable.getRegionsInfo()
-        Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
+         Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
         verifyTableRegions(regions.keySet());
       } finally {
         IOUtils.closeQuietly(table);

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 d9883e0..ac5e41e 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
@@ -37,13 +37,12 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -94,7 +93,8 @@ public class TestRegionMergeTransactionOnCluster {
   private static HMaster master;
   private static Admin admin;
 
-  static void setupOnce() throws Exception {
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
     // Start a cluster
     TEST_UTIL.startMiniCluster(NB_SERVERS);
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@@ -103,13 +103,6 @@ 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();
@@ -148,7 +141,7 @@ public class TestRegionMergeTransactionOnCluster {
     }
 
     // We should not be able to assign it again
-    am.assign(hri, true, true);
+    am.assign(hri, true);
     assertFalse("Merged region can't be assigned",
       regionStates.isRegionInTransition(hri));
     assertTrue(regionStates.isRegionInState(hri, State.MERGED));
@@ -162,6 +155,7 @@ public class TestRegionMergeTransactionOnCluster {
     table.close();
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testCleanMergeReference() throws Exception {
     LOG.info("Starting testCleanMergeReference");

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
index 9f64a7c..a7067f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
@@ -45,9 +45,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -94,26 +92,18 @@ public class TestRegionReplicas {
 
   @AfterClass
   public static void afterClass() throws Exception {
+    HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
     table.close();
     HTU.shutdownMiniCluster();
   }
 
-  @After
-  public void after() throws Exception {
-    // Clean the state if the test failed before cleaning the znode
-    // It does not manage all bad failures, so if there are multiple failures, only
-    //  the first one should be looked at.
-    ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriPrimary);
-  }
-
   private HRegionServer getRS() {
     return HTU.getMiniHBaseCluster().getRegionServer(0);
   }
 
   private void openRegion(HRegionInfo hri) throws Exception {
-    ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
     // first version is '0'
-    AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, 0, null, null);
+    AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, null, null);
     AdminProtos.OpenRegionResponse responseOpen = getRS().getRSRpcServices().openRegion(null, orr);
     Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
     Assert.assertTrue(responseOpen.getOpeningState(0).
@@ -122,16 +112,12 @@ public class TestRegionReplicas {
   }
 
   private void closeRegion(HRegionInfo hri) throws Exception {
-    ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-
     AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(getRS().getServerName(),
-        hri.getEncodedName(), true);
+        hri.getEncodedName());
     AdminProtos.CloseRegionResponse responseClose = getRS().getRSRpcServices().closeRegion(null, crr);
     Assert.assertTrue(responseClose.getClosed());
 
     checkRegionIsClosed(hri.getEncodedName());
-
-    ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), getRS().getServerName());
   }
 
   private void checkRegionIsOpened(String encodedRegionName) throws Exception {
@@ -141,9 +127,6 @@ public class TestRegionReplicas {
     }
 
     Assert.assertTrue(getRS().getRegionByEncodedName(encodedRegionName).isAvailable());
-
-    Assert.assertTrue(
-        ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), encodedRegionName, getRS().getServerName()));
   }
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 0843abe..3164f1d 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
@@ -26,29 +26,22 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
-import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -77,7 +70,6 @@ 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());
 
@@ -94,27 +86,37 @@ public class TestRegionServerNoMaster {
   }
 
   public static void stopMasterAndAssignMeta(HBaseTestingUtility HTU)
-      throws NodeExistsException, KeeperException, IOException, InterruptedException {
-    // No master
-    HTU.getHBaseCluster().getMaster().stopMaster();
+      throws IOException, InterruptedException {
+    // Stop master
+    HMaster master = HTU.getHBaseCluster().getMaster();
+    ServerName masterAddr = master.getServerName();
+    master.stopMaster();
 
     Log.info("Waiting until master thread exits");
     while (HTU.getHBaseCluster().getMasterThread() != null
         && HTU.getHBaseCluster().getMasterThread().isAlive()) {
       Threads.sleep(100);
     }
+
+    HRegionServer.TEST_SKIP_REPORTING_TRANSITION = true;
     // Master is down, so is the meta. We need to assign it somewhere
     // so that regions can be assigned during the mocking phase.
-    HRegionServer hrs = HTU.getHBaseCluster().getRegionServer(0);
+    HRegionServer hrs = HTU.getHBaseCluster()
+      .getLiveRegionServerThreads().get(0).getRegionServer();
     ZooKeeperWatcher zkw = hrs.getZooKeeper();
-    ZKAssign.createNodeOffline(
-      zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
+    MetaTableLocator mtl = new MetaTableLocator();
+    ServerName sn = mtl.getMetaRegionLocation(zkw);
+    if (sn != null && !masterAddr.equals(sn)) {
+      return;
+    }
+
     ProtobufUtil.openRegion(hrs.getRSRpcServices(),
       hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
-    MetaTableLocator mtl = new MetaTableLocator();
     while (true) {
-      ServerName sn = mtl.getMetaRegionLocation(zkw);
-      if (sn != null && sn.equals(hrs.getServerName())) {
+      sn = mtl.getMetaRegionLocation(zkw);
+      if (sn != null && sn.equals(hrs.getServerName())
+          && hrs.onlineRegions.containsKey(
+              HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
         break;
       }
       Thread.sleep(100);
@@ -135,19 +137,11 @@ public class TestRegionServerNoMaster {
 
   @AfterClass
   public static void afterClass() throws Exception {
+    HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
     table.close();
     HTU.shutdownMiniCluster();
   }
 
-  @After
-  public void after() throws Exception {
-    // Clean the state if the test failed before cleaning the znode
-    // It does not manage all bad failures, so if there are multiple failures, only
-    //  the first one should be looked at.
-    ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hri);
-  }
-
-
   private static HRegionServer getRS() {
     return HTU.getHBaseCluster().getLiveRegionServerThreads().get(0).getRegionServer();
   }
@@ -157,11 +151,8 @@ public class TestRegionServerNoMaster {
    * Reopen the region. Reused in multiple tests as we always leave the region open after a test.
    */
   private void reopenRegion() throws Exception {
-    // We reopen. We need a ZK node here, as a open is always triggered by a master.
-    ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-    // first version is '0'
     AdminProtos.OpenRegionRequest orr =
-      RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, 0, null, null);
+      RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, null, null);
     AdminProtos.OpenRegionResponse responseOpen = getRS().rpcServices.openRegion(null, orr);
     Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
     Assert.assertTrue(responseOpen.getOpeningState(0).
@@ -178,10 +169,6 @@ public class TestRegionServerNoMaster {
     }
 
     Assert.assertTrue(getRS().getRegion(regionName).isAvailable());
-
-    Assert.assertTrue(
-      ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
-        getRS().getServerName()));
   }
 
 
@@ -196,8 +183,6 @@ public class TestRegionServerNoMaster {
     } catch (NotServingRegionException expected) {
       // That's how it work: if the region is closed we have an exception.
     }
-
-    // We don't delete the znode here, because there is not always a znode.
   }
 
 
@@ -207,7 +192,7 @@ public class TestRegionServerNoMaster {
   private void closeNoZK() throws Exception {
     // no transition in ZK
     AdminProtos.CloseRegionRequest crr =
-        RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
+        RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName);
     AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
     Assert.assertTrue(responseClose.getClosed());
 
@@ -222,42 +207,6 @@ public class TestRegionServerNoMaster {
     reopenRegion();
   }
 
-  @Test(timeout = 60000)
-  public void testCloseByMasterWithoutZNode() throws Exception {
-
-    // Transition in ZK on. This should fail, as there is no znode
-    AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
-      getRS().getServerName(), regionName, true);
-    AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
-    Assert.assertTrue(responseClose.getClosed());
-
-    // now waiting. After a while, the transition should be done
-    while (!getRS().getRegionsInTransitionInRS().isEmpty()) {
-      Thread.sleep(1);
-    }
-
-    // the region is still available, the close got rejected at the end
-    Assert.assertTrue("The close should have failed", getRS().getRegion(regionName).isAvailable());
-  }
-
-  @Test(timeout = 60000)
-  public void testOpenCloseByMasterWithZNode() throws Exception {
-
-    ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-
-    AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
-      getRS().getServerName(), regionName, true);
-    AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
-    Assert.assertTrue(responseClose.getClosed());
-
-    checkRegionIsClosed();
-
-    ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
-      getRS().getServerName());
-
-    reopenRegion();
-  }
-
   /**
    * Test that we can send multiple openRegion to the region server.
    * This is used when:
@@ -275,13 +224,10 @@ public class TestRegionServerNoMaster {
     closeNoZK();
     checkRegionIsClosed();
 
-    // We reopen. We need a ZK node here, as a open is always triggered by a master.
-    ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-
     // We're sending multiple requests in a row. The region server must handle this nicely.
     for (int i = 0; i < 10; i++) {
       AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
-        getRS().getServerName(), hri, 0, null, null);
+        getRS().getServerName(), hri, null, null);
       AdminProtos.OpenRegionResponse responseOpen = getRS().rpcServices.openRegion(null, orr);
       Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
 
@@ -307,7 +253,7 @@ public class TestRegionServerNoMaster {
       // fake region to be closing now, need to clear state afterwards
       getRS().regionsInTransitionInRS.put(hri.getEncodedNameAsBytes(), Boolean.FALSE);
       AdminProtos.OpenRegionRequest orr =
-        RequestConverter.buildOpenRegionRequest(sn, hri, 0, null, null);
+        RequestConverter.buildOpenRegionRequest(sn, hri, null, null);
       getRS().rpcServices.openRegion(null, orr);
       Assert.fail("The closing region should not be opened");
     } catch (ServiceException se) {
@@ -320,12 +266,9 @@ public class TestRegionServerNoMaster {
 
   @Test(timeout = 60000)
   public void testMultipleCloseFromMaster() throws Exception {
-
-    // As opening, we must support multiple requests on the same region
-    ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
     for (int i = 0; i < 10; i++) {
       AdminProtos.CloseRegionRequest crr =
-          RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, 0, null, true);
+          RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, null);
       try {
         AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
         Assert.assertEquals("The first request should succeeds", 0, i);
@@ -338,11 +281,6 @@ public class TestRegionServerNoMaster {
 
     checkRegionIsClosed();
 
-    Assert.assertTrue(
-      ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
-        getRS().getServerName())
-    );
-
     reopenRegion();
   }
 
@@ -356,12 +294,11 @@ public class TestRegionServerNoMaster {
     checkRegionIsClosed();
 
     // Let do the initial steps, without having a handler
-    ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
     getRS().getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
 
     // That's a close without ZK.
     AdminProtos.CloseRegionRequest crr =
-        RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
+        RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName);
     try {
       getRS().rpcServices.closeRegion(null, crr);
       Assert.assertTrue(false);
@@ -375,89 +312,11 @@ public class TestRegionServerNoMaster {
     // Let's start the open handler
     HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
 
-    BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(getRS());
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(getRS().getServerName());
-    zkCrd.setVersionOfOfflineNode(0);
-
-    getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd,
-      csm.getOpenRegionCoordination(), zkCrd));
+    getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd));
 
     // The open handler should have removed the region from RIT but kept the region closed
     checkRegionIsClosed();
 
-    // The open handler should have updated the value in ZK.
-    Assert.assertTrue(ZKAssign.deleteNode(
-        getRS().getZooKeeper(), hri.getEncodedName(),
-        EventType.RS_ZK_REGION_FAILED_OPEN, 1)
-    );
-
-    reopenRegion();
-  }
-
-  /**
-   * Test an open then a close with ZK. This is going to mess-up the ZK states, so
-   * the opening will fail as well because it doesn't find what it expects in ZK.
-   */
-  @Test(timeout = 60000)
-  public void testCancelOpeningWithZK() throws Exception {
-    // We close
-    closeNoZK();
-    checkRegionIsClosed();
-
-    // Let do the initial steps, without having a handler
-    getRS().getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
-
-    // That's a close without ZK.
-    ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
-    AdminProtos.CloseRegionRequest crr =
-        RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
-    try {
-      getRS().rpcServices.closeRegion(null, crr);
-      Assert.assertTrue(false);
-    } catch (ServiceException expected) {
-      Assert.assertTrue(expected.getCause() instanceof NotServingRegionException);
-    }
-
-    // The close should have left the ZK state as it is: it's the job the AM to delete it
-    Assert.assertTrue(ZKAssign.deleteNode(
-        getRS().getZooKeeper(), hri.getEncodedName(),
-        EventType.M_ZK_REGION_CLOSING, 0)
-    );
-
-    // The state in RIT should have changed to close
-    Assert.assertEquals(Boolean.FALSE, getRS().getRegionsInTransitionInRS().get(
-        hri.getEncodedNameAsBytes()));
-
-    // Let's start the open handler
-    // It should not succeed for two reasons:
-    //  1) There is no ZK node
-    //  2) The region in RIT was changed.
-    // The order is more or less implementation dependant.
-    HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
-
-    BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-    csm.initialize(getRS());
-    csm.start();
-
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkCrd.setServerName(getRS().getServerName());
-    zkCrd.setVersionOfOfflineNode(0);
-
-    getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd,
-      csm.getOpenRegionCoordination(), zkCrd));
-
-    // The open handler should have removed the region from RIT but kept the region closed
-    checkRegionIsClosed();
-
-    // We should not find any znode here.
-    Assert.assertEquals(-1, ZKAssign.getVersion(HTU.getZooKeeperWatcher(), hri));
-
     reopenRegion();
   }
 
@@ -473,7 +332,7 @@ public class TestRegionServerNoMaster {
     ServerName earlierServerName = ServerName.valueOf(sn.getHostname(), sn.getPort(), 1);
 
     try {
-      CloseRegionRequest request = RequestConverter.buildCloseRegionRequest(earlierServerName, regionName, true);
+      CloseRegionRequest request = RequestConverter.buildCloseRegionRequest(earlierServerName, regionName);
       getRS().getRSRpcServices().closeRegion(null, request);
       Assert.fail("The closeRegion should have been rejected");
     } catch (ServiceException se) {
@@ -485,7 +344,7 @@ public class TestRegionServerNoMaster {
     closeNoZK();
     try {
       AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
-        earlierServerName, hri, 0, null, null);
+        earlierServerName, hri, null, null);
       getRS().getRSRpcServices().openRegion(null, orr);
       Assert.fail("The openRegion should have been rejected");
     } catch (ServiceException se) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 da4b3d9..3818b60 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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -48,15 +47,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -68,14 +66,10 @@ 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.client.TestReplicasClient.SlowMeCopro;
-import org.apache.hadoop.hbase.coordination.ZKSplitTransactionCoordination;
-import org.apache.hadoop.hbase.coordination.ZkCloseRegionCoordination;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
@@ -90,12 +84,9 @@ import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -120,26 +111,15 @@ public class TestSplitTransactionOnCluster {
   private static final int NB_SERVERS = 3;
   private static CountDownLatch latch = new CountDownLatch(1);
   private static volatile boolean secondSplit = false;
-  private static volatile boolean callRollBack = false;
-  private static volatile boolean firstSplitCompleted = false;
-  private static boolean useZKForAssignment = true;
 
   static final HBaseTestingUtility TESTING_UTIL =
     new HBaseTestingUtility();
 
-  static void setupOnce() throws Exception {
+  @BeforeClass public static void before() throws Exception {
     TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
-    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();
   }
@@ -181,100 +161,6 @@ public class TestSplitTransactionOnCluster {
     return hri;
   }
 
-  @SuppressWarnings("deprecation")
-  @Test(timeout = 60000)
-  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"));
-      final List<HRegion> regions = cluster.getRegions(tableName);
-      HRegionInfo hri = getAndCheckSingleTableRegion(regions);
-      int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
-      final HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
-      insertData(tableName.getName(), admin, t);
-      t.close();
-
-      // 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.
-      cluster.getMaster().setCatalogJanitorEnabled(false);
-
-      // find a splittable region
-      final HRegion region = findSplittableRegion(regions);
-      assertTrue("not able to find a splittable region", region != null);
-      MockedCoordinatedStateManager cp = new MockedCoordinatedStateManager();
-      cp.initialize(regionServer, region);
-      cp.start();
-      regionServer.csm = cp;
-
-      new Thread() {
-        @Override
-        public void run() {
-          SplitTransaction st = null;
-          st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
-          try {
-            st.prepare();
-            st.execute(regionServer, regionServer);
-          } catch (IOException e) {
-
-          }
-        }
-      }.start();
-      for (int i = 0; !callRollBack && i < 100; i++) {
-        Thread.sleep(100);
-      }
-      assertTrue("Waited too long for rollback", callRollBack);
-      SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row3"));
-      try {
-        secondSplit = true;
-        // make region splittable
-        region.initialize();
-        st.prepare();
-        st.execute(regionServer, regionServer);
-      } catch (IOException e) {
-        LOG.debug("Rollback started :"+ e.getMessage());
-        st.rollback(regionServer, regionServer);
-      }
-      for (int i=0; !firstSplitCompleted && i<100; i++) {
-        Thread.sleep(100);
-      }
-      assertTrue("fist split did not complete", firstSplitCompleted);
-
-      RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
-      Map<String, RegionState> rit = regionStates.getRegionsInTransition();
-
-      for (int i=0; rit.containsKey(hri.getTable()) && i<100; i++) {
-        Thread.sleep(100);
-      }
-      assertFalse("region still in transition", rit.containsKey(
-          rit.containsKey(hri.getTable())));
-
-      List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
-      // Region server side split is successful.
-      assertEquals("The parent region should be splitted", 2, onlineRegions.size());
-      //Should be present in RIT
-      List<HRegionInfo> regionsOfTable = cluster.getMaster().getAssignmentManager()
-          .getRegionStates().getRegionsOfTable(tableName);
-      // Master side should also reflect the same
-      assertEquals("No of regions in master", 2, regionsOfTable.size());
-    } finally {
-      admin.setBalancerRunning(true, false);
-      secondSplit = false;
-      firstSplitCompleted = false;
-      callRollBack = false;
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-      TESTING_UTIL.deleteTable(tableName);
-    }
-  }
-
   @Test(timeout = 60000)
   public void testRITStateForRollback() throws Exception {
     final TableName tableName =
@@ -382,20 +268,15 @@ public class TestSplitTransactionOnCluster {
 
  /**
    * A test that intentionally has master fail the processing of the split message.
-   * Tests that the regionserver split ephemeral node gets cleaned up if it
-   * crashes and that after we process server shutdown, the daughters are up on
-   * line.
+   * Tests that after we process server shutdown, the daughters are up on line.
    * @throws IOException
    * @throws InterruptedException
-   * @throws NodeExistsException
-   * @throws KeeperException
-   * @throws DeserializationException
+   * @throws ServiceException
    */
-  @Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
-  throws IOException, InterruptedException, NodeExistsException, KeeperException,
-      DeserializationException, ServiceException {
+  @Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling()
+  throws IOException, InterruptedException, ServiceException {
     final byte [] tableName =
-      Bytes.toBytes("testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling");
+      Bytes.toBytes("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
 
     // Create table then get the single region for our new table.
     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@@ -418,48 +299,15 @@ public class TestSplitTransactionOnCluster {
       // Now, before we split, set special flag in master, a flag that has
       // it FAIL the processing of split.
       AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
-      // Now try splitting and it should work.
-      split(hri, server, regionCount);
-
-      String path = ZKAssign.getNodeName(TESTING_UTIL.getZooKeeperWatcher(),
-        hri.getEncodedName());
-      RegionTransition rt = null;
-      Stat stats = null;
-      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, for ZK-less assignment, the server is auto aborted
-        cluster.abortRegionServer(tableRegionIndex);
+      try {
+        // Now try splitting and it should work.
+        split(hri, server, regionCount);
+      } catch (RegionServerStoppedException rsse) {
+        // Expected. The regionserver should crash
       }
+
       waitUntilRegionServerDead();
       awaitDaughters(tableName, 2);
-      if (useZKForAssignment) {
-        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);
-        }
-        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;
@@ -495,15 +343,8 @@ public class TestSplitTransactionOnCluster {
       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
       printOutRegions(server, "Initial regions: ");
       int regionCount = ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
-      // 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);
-      if (useZKForAssignment) {
-        ZKAssign.createNodeClosing(TESTING_UTIL.getZooKeeperWatcher(),
-          hri, fakedServer);
-      } else {
-        regionStates.updateRegionState(hri, RegionState.State.CLOSING);
-      }
+      regionStates.updateRegionState(hri, RegionState.State.CLOSING);
+
       // Now try splitting.... should fail.  And each should successfully
       // rollback.
       this.admin.split(hri.getRegionNameAsString());
@@ -515,13 +356,8 @@ public class TestSplitTransactionOnCluster {
         assertEquals(regionCount, ProtobufUtil.getOnlineRegions(
           server.getRSRpcServices()).size());
       }
-      if (useZKForAssignment) {
-        // Now clear the zknode
-        ZKAssign.deleteClosingNode(TESTING_UTIL.getZooKeeperWatcher(),
-          hri, fakedServer);
-      } else {
-        regionStates.regionOnline(hri, server.getServerName());
-      }
+      regionStates.regionOnline(hri, server.getServerName());
+
       // Now try splitting and it should work.
       split(hri, server, regionCount);
       // Get daughters
@@ -541,7 +377,7 @@ public class TestSplitTransactionOnCluster {
    * @throws InterruptedException
    */
   @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
-  throws IOException, InterruptedException, ServiceException {
+  throws IOException, InterruptedException {
     final byte [] tableName =
       Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit");
 
@@ -698,103 +534,6 @@ public class TestSplitTransactionOnCluster {
   }
 
   /**
-   * Verifies HBASE-5806.  When splitting is partially done and the master goes down
-   * when the SPLIT node is in either SPLIT or SPLITTING state.
-   *
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws NodeExistsException
-   * @throws KeeperException
-   * @throws DeserializationException
-   */
-  @Test(timeout = 400000)
-  public void testMasterRestartWhenSplittingIsPartial()
-      throws IOException, InterruptedException, NodeExistsException,
-      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);
-    HRegionInfo hri = getAndCheckSingleTableRegion(regions);
-
-    int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
-
-    // 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.
-    cluster.getMaster().setCatalogJanitorEnabled(false);
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(t.getConfiguration(),
-      "testMasterRestartWhenSplittingIsPartial", new UselessTestAbortable());
-    try {
-      // Add a bit of load up into the table so splittable.
-      TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY, false);
-      // Get region pre-split.
-      HRegionServer server = cluster.getRegionServer(tableRegionIndex);
-      printOutRegions(server, "Initial regions: ");
-      // Now, before we split, set special flag in master, a flag that has
-      // it FAIL the processing of split.
-      AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
-      // Now try splitting and it should work.
-
-      this.admin.split(hri.getRegionNameAsString());
-      checkAndGetDaughters(tableName);
-      // Assert the ephemeral node is up in zk.
-      String path = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-      Stat stats = zkw.getRecoverableZooKeeper().exists(path, false);
-      LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
-          + stats);
-      byte[] bytes = ZKAssign.getData(zkw, hri.getEncodedName());
-      RegionTransition rtd = RegionTransition.parseFrom(bytes);
-      // State could be SPLIT or SPLITTING.
-      assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)
-          || rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
-
-      // abort and wait for new master.
-      MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
-
-      this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
-
-      // Update the region to be offline and split, so that HRegionInfo#equals
-      // returns true in checking rebuilt region states map.
-      hri.setOffline(true);
-      hri.setSplit(true);
-      ServerName regionServerOfRegion = master.getAssignmentManager()
-        .getRegionStates().getRegionServerOfRegion(hri);
-      assertTrue(regionServerOfRegion != null);
-
-      // Remove the block so that split can move ahead.
-      AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
-      String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-      Stat stat = new Stat();
-      byte[] data = ZKUtil.getDataNoWatch(zkw, node, stat);
-      // ZKUtil.create
-      for (int i=0; data != null && i<60; i++) {
-        Thread.sleep(1000);
-        data = ZKUtil.getDataNoWatch(zkw, node, stat);
-      }
-      assertNull("Waited too long for ZK node to be removed: "+node, data);
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      assertTrue("Split parent should be in SPLIT state",
-        regionStates.isRegionInState(hri, State.SPLIT));
-      regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
-      assertTrue(regionServerOfRegion == null);
-    } finally {
-      // Set this flag back.
-      AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
-      admin.setBalancerRunning(true, false);
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-      t.close();
-      zkw.close();
-    }
-  }
-
-  /**
    * Verifies HBASE-5806.  Here the case is that splitting is completed but before the
    * CJ could remove the parent region the master is killed and restarted.
    * @throws IOException
@@ -830,20 +569,6 @@ public class TestSplitTransactionOnCluster {
 
       this.admin.split(hri.getRegionNameAsString());
       checkAndGetDaughters(tableName);
-      // Assert the ephemeral node is up in zk.
-      String path = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-      Stat stats = zkw.getRecoverableZooKeeper().exists(path, false);
-      LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
-          + stats);
-      String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-      Stat stat = new Stat();
-      byte[] data = ZKUtil.getDataNoWatch(zkw, node, stat);
-      // ZKUtil.create
-      for (int i=0; data != null && i<60; i++) {
-        Thread.sleep(1000);
-        data = ZKUtil.getDataNoWatch(zkw, node, stat);
-      }
-      assertNull("Waited too long for ZK node to be removed: "+node, data);
 
       MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
 
@@ -885,7 +610,6 @@ public class TestSplitTransactionOnCluster {
 
   @Test(timeout = 60000)
   public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTING_UTIL);
     final TableName tableName =
         TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
     // Create table then get the single region for our new table.
@@ -916,11 +640,6 @@ public class TestSplitTransactionOnCluster {
         tableName);
       assertEquals("The specified table should present.", true, tableExists);
     } finally {
-      if (regions != null) {
-        String node = ZKAssign.getNodeName(zkw, regions.get(0).getRegionInfo()
-            .getEncodedName());
-        ZKUtil.deleteNodeFailSilent(zkw, node);
-      }
       admin.setBalancerRunning(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
@@ -929,7 +648,6 @@ public class TestSplitTransactionOnCluster {
 
   @Test
   public void testSplitWithRegionReplicas() throws Exception {
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTING_UTIL);
     final TableName tableName =
         TableName.valueOf("foobar");
     HTableDescriptor htd = TESTING_UTIL.createTableDescriptor("foobar");
@@ -938,7 +656,6 @@ public class TestSplitTransactionOnCluster {
     // Create table then get the single region for our new table.
     HTable t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")},
         TESTING_UTIL.getConfiguration());
-    int count;
     List<HRegion> oldRegions;
     do {
       oldRegions = cluster.getRegions(tableName);
@@ -960,9 +677,6 @@ public class TestSplitTransactionOnCluster {
       regionServerIndex = cluster.getServerWith(region.getRegionName());
       regionServer = cluster.getRegionServer(regionServerIndex);
       assertTrue("not able to find a splittable region", region != null);
-      String node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
-          region.getRegionInfo().getEncodedName());
-      regionServer.getZooKeeper().sync(node);
       SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2"));
       try {
         st.prepare();
@@ -1093,7 +807,7 @@ public class TestSplitTransactionOnCluster {
       }
 
       // We should not be able to assign it again
-      am.assign(hri, true, true);
+      am.assign(hri, true);
       assertFalse("Split region can't be assigned",
         regionStates.isRegionInTransition(hri));
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
@@ -1176,17 +890,6 @@ public class TestSplitTransactionOnCluster {
           throw new SplittingNodeCreationFailedException ();
         }
       };
-      String node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
-          region.getRegionInfo().getEncodedName());
-      regionServer.getZooKeeper().sync(node);
-      for (int i = 0; i < 100; i++) {
-        // We expect the znode to be deleted by this time. Here the
-        // znode could be in OPENED state and the
-        // master has not yet deleted the znode.
-        if (ZKUtil.checkExists(regionServer.getZooKeeper(), node) != -1) {
-          Thread.sleep(100);
-        }
-      }
       try {
         st.prepare();
         st.execute(regionServer, regionServer);
@@ -1196,13 +899,7 @@ public class TestSplitTransactionOnCluster {
         // This will at least make the test to fail;
         assertTrue("Should be instance of CreateSplittingNodeFailedException",
             e instanceof SplittingNodeCreationFailedException );
-        node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
-            region.getRegionInfo().getEncodedName());
-        {
-          assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
-        }
         assertTrue(st.rollback(regionServer, regionServer));
-        assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
       }
     } finally {
       TESTING_UTIL.deleteTable(tableName);
@@ -1213,8 +910,6 @@ public class TestSplitTransactionOnCluster {
         public void initialize(Server server, HRegion region) {
           this.server = server;
           this.watcher = server.getZooKeeper();
-          splitTransactionCoordination = new MockedSplitTransactionCoordination(this, watcher, region);
-          closeRegionCoordination = new ZkCloseRegionCoordination(this, watcher);
         }
       }
 
@@ -1237,41 +932,7 @@ public class TestSplitTransactionOnCluster {
           }
           return super.rollback(server, services);
         }
-
-
-      }
-
-  public static class MockedSplitTransactionCoordination extends ZKSplitTransactionCoordination {
-
-    private HRegion currentRegion;
-
-    public MockedSplitTransactionCoordination(CoordinatedStateManager coordinationProvider,
-        ZooKeeperWatcher watcher, HRegion region) {
-      super(coordinationProvider, watcher);
-      currentRegion = region;
-    }
-
-    @Override
-    public void completeSplitTransaction(RegionServerServices services, HRegion a, HRegion b,
-        SplitTransactionDetails std, HRegion parent) throws IOException {
-      if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
-          .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
-        try {
-          if (!secondSplit){
-            callRollBack = true;
-            latch.await();
-          }
-        } catch (InterruptedException e) {
-        }
-
       }
-      super.completeSplitTransaction(services, a, b, std, parent);
-      if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
-          .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
-        firstSplitCompleted = true;
-      }
-    }
-  }
 
   private HRegion findSplittableRegion(final List<HRegion> regions) throws InterruptedException {
     for (int i = 0; i < 5; ++i) {
@@ -1313,22 +974,14 @@ public class TestSplitTransactionOnCluster {
   private void split(final HRegionInfo hri, final HRegionServer server, final int regionCount)
       throws IOException, InterruptedException {
     this.admin.split(hri.getRegionNameAsString());
-    try {
-      for (int i = 0; ProtobufUtil.getOnlineRegions(
-          server.getRSRpcServices()).size() <= regionCount && i < 300; i++) {
-        LOG.debug("Waiting on region to split");
-        Thread.sleep(100);
-      }
-
-      assertFalse("Waited too long for split",
-        ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size() <= regionCount);
-    } catch (RegionServerStoppedException e) {
-      if (useZKForAssignment) {
-        // If not using ZK for assignment, the exception may be expected.
-        LOG.error(e);
-        throw e;
-      }
+    for (int i = 0; ProtobufUtil.getOnlineRegions(
+        server.getRSRpcServices()).size() <= regionCount && i < 300; i++) {
+      LOG.debug("Waiting on region to split");
+      Thread.sleep(100);
     }
+
+    assertFalse("Waited too long for split",
+      ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size() <= regionCount);
   }
 
   /**
@@ -1411,13 +1064,14 @@ public class TestSplitTransactionOnCluster {
 
   private void waitUntilRegionServerDead() throws InterruptedException, InterruptedIOException {
     // Wait until the master processes the RS shutdown
-    for (int i=0; cluster.getMaster().getClusterStatus().
-        getServers().size() > NB_SERVERS && i<100; i++) {
+    for (int i=0; (cluster.getMaster().getClusterStatus().getServers().size() > NB_SERVERS
+        || cluster.getLiveRegionServerThreads().size() > NB_SERVERS) && i<100; i++) {
       LOG.info("Waiting on server to go down");
       Thread.sleep(100);
     }
-    assertFalse("Waited too long for RS to die", cluster.getMaster().getClusterStatus().
-        getServers().size() > NB_SERVERS);
+    assertFalse("Waited too long for RS to die",
+      cluster.getMaster().getClusterStatus(). getServers().size() > NB_SERVERS
+        || cluster.getLiveRegionServerThreads().size() > NB_SERVERS);
   }
 
   private void awaitDaughters(byte[] tableName, int numDaughters) throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index d9e8480..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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/17dff681/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
deleted file mode 100644
index 1e3de64..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- *
- * 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();
-  }
-}
-


[03/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 5f1159a..5c35611 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
@@ -19,23 +19,17 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -44,34 +38,17 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 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.TableStateManager;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 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;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.data.Stat;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -79,877 +56,6 @@ import org.junit.experimental.categories.Category;
 public class TestMasterFailover {
   private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
 
-  /**
-   * Complex test of master failover that tests as many permutations of the
-   * different possible states that regions in transition could be in within ZK.
-   * <p>
-   * This tests the proper handling of these states by the failed-over master
-   * and includes a thorough testing of the timeout code as well.
-   * <p>
-   * Starts with a single master and three regionservers.
-   * <p>
-   * Creates two tables, enabledTable and disabledTable, each containing 5
-   * regions.  The disabledTable is then disabled.
-   * <p>
-   * After reaching steady-state, the master is killed.  We then mock several
-   * states in ZK.
-   * <p>
-   * After mocking them, we will startup a new master which should become the
-   * active master and also detect that it is a failover.  The primary test
-   * passing condition will be that all regions of the enabled table are
-   * assigned and all the regions of the disabled table are not assigned.
-   * <p>
-   * The different scenarios to be tested are below:
-   * <p>
-   * <b>ZK State:  OFFLINE</b>
-   * <p>A node can get into OFFLINE state if</p>
-   * <ul>
-   * <li>An RS fails to open a region, so it reverts the state back to OFFLINE
-   * <li>The Master is assigning the region to a RS before it sends RPC
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Master has assigned an enabled region but RS failed so a region is
-   *     not assigned anywhere and is sitting in ZK as OFFLINE</li>
-   * <li>This seems to cover both cases?</li>
-   * </ul>
-   * <p>
-   * <b>ZK State:  CLOSING</b>
-   * <p>A node can get into CLOSING state if</p>
-   * <ul>
-   * <li>An RS has begun to close a region
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region of enabled table was being closed but did not complete
-   * <li>Region of disabled table was being closed but did not complete
-   * </ul>
-   * <p>
-   * <b>ZK State:  CLOSED</b>
-   * <p>A node can get into CLOSED state if</p>
-   * <ul>
-   * <li>An RS has completed closing a region but not acknowledged by master yet
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region of a table that should be enabled was closed on an RS
-   * <li>Region of a table that should be disabled was closed on an RS
-   * </ul>
-   * <p>
-   * <b>ZK State:  OPENING</b>
-   * <p>A node can get into OPENING state if</p>
-   * <ul>
-   * <li>An RS has begun to open a region
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>RS was opening a region of enabled table but never finishes
-   * </ul>
-   * <p>
-   * <b>ZK State:  OPENED</b>
-   * <p>A node can get into OPENED state if</p>
-   * <ul>
-   * <li>An RS has finished opening a region but not acknowledged by master yet
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region of a table that should be enabled was opened on an RS
-   * <li>Region of a table that should be disabled was opened on an RS
-   * </ul>
-   * @throws Exception
-   */
-  @Test (timeout=240000)
-  public void testMasterFailoverWithMockedRIT() throws Exception {
-
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 3;
-
-    // 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);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    log("Cluster started");
-
-    // Create a ZKW to use in the test
-    ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-
-    // 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());
-
-    // disable load balancing on this master
-    master.balanceSwitch(false);
-
-    // create two tables in META, each with 10 regions
-    byte [] FAMILY = Bytes.toBytes("family");
-    byte [][] SPLIT_KEYS = new byte [][] {
-        new byte[0], Bytes.toBytes("aaa"), Bytes.toBytes("bbb"),
-        Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
-        Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
-        Bytes.toBytes("iii"), Bytes.toBytes("jjj")
-    };
-
-    byte [] enabledTable = Bytes.toBytes("enabledTable");
-    HTableDescriptor htdEnabled = new HTableDescriptor(TableName.valueOf(enabledTable));
-    htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
-
-    FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = FSUtils.getRootDir(conf);
-    FSTableDescriptors fstd = new FSTableDescriptors(filesystem, rootdir);
-    // Write the .tableinfo
-    fstd.createTableDescriptor(htdEnabled);
-
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getTableName(), null, null);
-    createRegion(hriEnabled, rootdir, conf, htdEnabled);
-
-    List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
-        TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
-
-    TableName disabledTable = TableName.valueOf("disabledTable");
-    HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
-    htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
-    // Write the .tableinfo
-    fstd.createTableDescriptor(htdDisabled);
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getTableName(), null, null);
-    createRegion(hriDisabled, rootdir, conf, htdDisabled);
-    List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
-        TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
-
-    TableName tableWithMergingRegions = TableName.valueOf("tableWithMergingRegions");
-    TEST_UTIL.createTable(tableWithMergingRegions, FAMILY, new byte [][] {Bytes.toBytes("m")});
-
-    log("Regions in hbase:meta and namespace have been created");
-
-    // at this point we only expect 4 regions to be assigned out
-    // (catalogs and namespace, + 2 merging regions)
-    assertEquals(4, cluster.countServedRegions());
-
-    // Move merging regions to the same region server
-    AssignmentManager am = master.getAssignmentManager();
-    RegionStates regionStates = am.getRegionStates();
-    List<HRegionInfo> mergingRegions = regionStates.getRegionsOfTable(tableWithMergingRegions);
-    assertEquals(2, mergingRegions.size());
-    HRegionInfo a = mergingRegions.get(0);
-    HRegionInfo b = mergingRegions.get(1);
-    HRegionInfo newRegion = RegionMergeTransaction.getMergedRegionInfo(a, b);
-    ServerName mergingServer = regionStates.getRegionServerOfRegion(a);
-    ServerName serverB = regionStates.getRegionServerOfRegion(b);
-    if (!serverB.equals(mergingServer)) {
-      RegionPlan plan = new RegionPlan(b, serverB, mergingServer);
-      am.balance(plan);
-      assertTrue(am.waitForAssignment(b));
-    }
-
-    // Let's just assign everything to first RS
-    HRegionServer hrs = cluster.getRegionServer(0);
-    ServerName serverName = hrs.getServerName();
-    HRegionInfo closingRegion = enabledRegions.remove(0);
-    // we'll need some regions to already be assigned out properly on live RS
-    List<HRegionInfo> enabledAndAssignedRegions = new ArrayList<HRegionInfo>();
-    enabledAndAssignedRegions.add(enabledRegions.remove(0));
-    enabledAndAssignedRegions.add(enabledRegions.remove(0));
-    enabledAndAssignedRegions.add(closingRegion);
-
-    List<HRegionInfo> disabledAndAssignedRegions = new ArrayList<HRegionInfo>();
-    disabledAndAssignedRegions.add(disabledRegions.remove(0));
-    disabledAndAssignedRegions.add(disabledRegions.remove(0));
-
-    // now actually assign them
-    for (HRegionInfo hri : enabledAndAssignedRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, serverName));
-      master.assignRegion(hri);
-    }
-    for (HRegionInfo hri : disabledAndAssignedRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, serverName));
-      master.assignRegion(hri);
-    }
-
-    // wait for no more RIT
-    log("Waiting for assignment to finish");
-    ZKAssign.blockUntilNoRIT(zkw);
-    log("Assignment completed");
-
-    // Stop the master
-    log("Aborting master");
-    cluster.abortMaster(0);
-    cluster.waitOnMaster(0);
-    log("Master has aborted");
-
-    /*
-     * Now, let's start mocking up some weird states as described in the method
-     * javadoc.
-     */
-
-    // Master is down, so is the meta. We need to assign it somewhere
-    // so that regions can be assigned during the mocking phase.
-    ZKAssign.createNodeOffline(
-      zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(),
-      hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
-
-    MetaTableLocator mtl = new MetaTableLocator();
-    while (true) {
-      ServerName sn = mtl.getMetaRegionLocation(zkw);
-      if (sn != null && sn.equals(hrs.getServerName())) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-
-    List<HRegionInfo> regionsThatShouldBeOnline = new ArrayList<HRegionInfo>();
-    List<HRegionInfo> regionsThatShouldBeOffline = new ArrayList<HRegionInfo>();
-
-    log("Beginning to mock scenarios");
-
-    // Disable the disabledTable in ZK
-    TableStateManager zktable = new ZKTableStateManager(zkw);
-    zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
-
-    /*
-     *  ZK = OFFLINE
-     */
-
-    // Region that should be assigned but is not and is in ZK as OFFLINE
-    // Cause: This can happen if the master crashed after creating the znode but before sending the
-    //  request to the region server
-    HRegionInfo region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, serverName);
-
-    /*
-     * ZK = CLOSING
-     */
-    // Cause: Same as offline.
-    regionsThatShouldBeOnline.add(closingRegion);
-    ZKAssign.createNodeClosing(zkw, closingRegion, serverName);
-
-    /*
-     * ZK = CLOSED
-     */
-
-    // Region of enabled table closed but not ack
-    //Cause: Master was down while the region server updated the ZK status.
-    region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    int version = ZKAssign.createNodeClosing(zkw, region, serverName);
-    ZKAssign.transitionNodeClosed(zkw, region, serverName, version);
-
-    // Region of disabled table closed but not ack
-    region = disabledRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    version = ZKAssign.createNodeClosing(zkw, region, serverName);
-    ZKAssign.transitionNodeClosed(zkw, region, serverName, version);
-
-    /*
-     * ZK = OPENED
-     */
-
-    // Region of enabled table was opened on RS
-    // Cause: as offline
-    region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, serverName);
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-
-    // Region of disable table was opened on RS
-    // Cause: Master failed while updating the status for this region server.
-    region = disabledRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, serverName);
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-
-    /*
-     * ZK = MERGING
-     */
-
-    // Regions of table of merging regions
-    // Cause: Master was down while merging was going on
-    ((BaseCoordinatedStateManager) hrs.getCoordinatedStateManager())
-      .getRegionMergeCoordination().startRegionMergeTransaction(newRegion, mergingServer, a, b);
-
-    /*
-     * ZK = NONE
-     */
-
-    /*
-     * DONE MOCKING
-     */
-
-    log("Done mocking data up in ZK");
-
-    // 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");
-
-    // Get new region states since master restarted
-    regionStates = master.getAssignmentManager().getRegionStates();
-    // Merging region should remain merging
-    assertTrue(regionStates.isRegionInState(a, State.MERGING));
-    assertTrue(regionStates.isRegionInState(b, State.MERGING));
-    assertTrue(regionStates.isRegionInState(newRegion, State.MERGING_NEW));
-    // Now remove the faked merging znode, merging regions should be
-    // offlined automatically, otherwise it is a bug in AM.
-    ZKAssign.deleteNodeFailSilent(zkw, newRegion);
-
-    // Failover should be completed, now wait for no RIT
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-    log("No more RIT in ZK, now doing final test verification");
-
-    // Grab all the regions that are online across RSs
-    Set<HRegionInfo> onlineRegions = new TreeSet<HRegionInfo>();
-    for (JVMClusterUtil.RegionServerThread rst :
-      cluster.getRegionServerThreads()) {
-      onlineRegions.addAll(ProtobufUtil.getOnlineRegions(
-        rst.getRegionServer().getRSRpcServices()));
-    }
-
-    // Now, everything that should be online should be online
-    for (HRegionInfo hri : regionsThatShouldBeOnline) {
-      assertTrue(onlineRegions.contains(hri));
-    }
-
-    // Everything that should be offline should not be online
-    for (HRegionInfo hri : regionsThatShouldBeOffline) {
-      if (onlineRegions.contains(hri)) {
-       LOG.debug(hri);
-      }
-      assertFalse(onlineRegions.contains(hri));
-    }
-
-    log("Done with verification, all passed, shutting down cluster");
-
-    // Done, shutdown the cluster
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Complex test of master failover that tests as many permutations of the
-   * different possible states that regions in transition could be in within ZK
-   * pointing to an RS that has died while no master is around to process it.
-   * <p>
-   * This tests the proper handling of these states by the failed-over master
-   * and includes a thorough testing of the timeout code as well.
-   * <p>
-   * Starts with a single master and two regionservers.
-   * <p>
-   * Creates two tables, enabledTable and disabledTable, each containing 5
-   * regions.  The disabledTable is then disabled.
-   * <p>
-   * After reaching steady-state, the master is killed.  We then mock several
-   * states in ZK.  And one of the RS will be killed.
-   * <p>
-   * After mocking them and killing an RS, we will startup a new master which
-   * should become the active master and also detect that it is a failover.  The
-   * primary test passing condition will be that all regions of the enabled
-   * table are assigned and all the regions of the disabled table are not
-   * assigned.
-   * <p>
-   * The different scenarios to be tested are below:
-   * <p>
-   * <b>ZK State:  CLOSING</b>
-   * <p>A node can get into CLOSING state if</p>
-   * <ul>
-   * <li>An RS has begun to close a region
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region was being closed but the RS died before finishing the close
-   * </ul>
-   * <b>ZK State:  OPENED</b>
-   * <p>A node can get into OPENED state if</p>
-   * <ul>
-   * <li>An RS has finished opening a region but not acknowledged by master yet
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region of a table that should be enabled was opened by a now-dead RS
-   * <li>Region of a table that should be disabled was opened by a now-dead RS
-   * </ul>
-   * <p>
-   * <b>ZK State:  NONE</b>
-   * <p>A region could not have a transition node if</p>
-   * <ul>
-   * <li>The server hosting the region died and no master processed it
-   * </ul>
-   * <p>We will mock the scenarios</p>
-   * <ul>
-   * <li>Region of enabled table was on a dead RS that was not yet processed
-   * <li>Region of disabled table was on a dead RS that was not yet processed
-   * </ul>
-   * @throws Exception
-   */
-  @Test (timeout=180000)
-  public void testMasterFailoverWithMockedRITOnDeadRS() throws Exception {
-
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 2;
-
-    // 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);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    log("Cluster started");
-
-    // Create a ZKW to use in the test
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-        "unittest", new Abortable() {
-
-          @Override
-          public void abort(String why, Throwable e) {
-            LOG.error("Fatal ZK Error: " + why, e);
-            org.junit.Assert.assertFalse("Fatal ZK error", true);
-          }
-
-          @Override
-          public boolean isAborted() {
-            return false;
-          }
-
-    });
-
-    // 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());
-
-    // disable load balancing on this master
-    master.balanceSwitch(false);
-
-    // create two tables in META, each with 30 regions
-    byte [] FAMILY = Bytes.toBytes("family");
-    byte[][] SPLIT_KEYS =
-        TEST_UTIL.getRegionSplitStartKeys(Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 30);
-
-    byte [] enabledTable = Bytes.toBytes("enabledTable");
-    HTableDescriptor htdEnabled = new HTableDescriptor(TableName.valueOf(enabledTable));
-    htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
-    FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = FSUtils.getRootDir(conf);
-    FSTableDescriptors fstd = new FSTableDescriptors(filesystem, rootdir);
-    // Write the .tableinfo
-    fstd.createTableDescriptor(htdEnabled);
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getTableName(),
-        null, null);
-    createRegion(hriEnabled, rootdir, conf, htdEnabled);
-
-    List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
-        TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
-
-    TableName disabledTable =
-        TableName.valueOf("disabledTable");
-    HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
-    htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
-    // Write the .tableinfo
-    fstd.createTableDescriptor(htdDisabled);
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getTableName(), null, null);
-    createRegion(hriDisabled, rootdir, conf, htdDisabled);
-
-    List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
-        TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
-
-    log("Regions in hbase:meta and Namespace have been created");
-
-    // at this point we only expect 2 regions to be assigned out (catalogs and namespace  )
-    assertEquals(2, cluster.countServedRegions());
-
-    // The first RS will stay online
-    List<RegionServerThread> regionservers =
-      cluster.getRegionServerThreads();
-    HRegionServer hrs = regionservers.get(0).getRegionServer();
-
-    // The second RS is going to be hard-killed
-    RegionServerThread hrsDeadThread = regionservers.get(1);
-    HRegionServer hrsDead = hrsDeadThread.getRegionServer();
-    ServerName deadServerName = hrsDead.getServerName();
-
-    // we'll need some regions to already be assigned out properly on live RS
-    List<HRegionInfo> enabledAndAssignedRegions = new ArrayList<HRegionInfo>();
-    enabledAndAssignedRegions.addAll(enabledRegions.subList(0, 6));
-    enabledRegions.removeAll(enabledAndAssignedRegions);
-    List<HRegionInfo> disabledAndAssignedRegions = new ArrayList<HRegionInfo>();
-    disabledAndAssignedRegions.addAll(disabledRegions.subList(0, 6));
-    disabledRegions.removeAll(disabledAndAssignedRegions);
-
-    // now actually assign them
-    for (HRegionInfo hri : enabledAndAssignedRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, hrs.getServerName()));
-      master.assignRegion(hri);
-    }
-    for (HRegionInfo hri : disabledAndAssignedRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, hrs.getServerName()));
-      master.assignRegion(hri);
-    }
-
-    log("Waiting for assignment to finish");
-    ZKAssign.blockUntilNoRIT(zkw);
-    master.assignmentManager.waitUntilNoRegionsInTransition(60000);
-    log("Assignment completed");
-
-    assertTrue(" Table must be enabled.", master.getAssignmentManager()
-        .getTableStateManager().isTableState(TableName.valueOf("enabledTable"),
-        ZooKeeperProtos.Table.State.ENABLED));
-    // we also need regions assigned out on the dead server
-    List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
-    enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6));
-    enabledRegions.removeAll(enabledAndOnDeadRegions);
-    List<HRegionInfo> disabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
-    disabledAndOnDeadRegions.addAll(disabledRegions.subList(0, 6));
-    disabledRegions.removeAll(disabledAndOnDeadRegions);
-
-    // set region plan to server to be killed and trigger assign
-    for (HRegionInfo hri : enabledAndOnDeadRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, deadServerName));
-      master.assignRegion(hri);
-    }
-    for (HRegionInfo hri : disabledAndOnDeadRegions) {
-      master.assignmentManager.regionPlans.put(hri.getEncodedName(),
-          new RegionPlan(hri, null, deadServerName));
-      master.assignRegion(hri);
-    }
-
-    // wait for no more RIT
-    log("Waiting for assignment to finish");
-    ZKAssign.blockUntilNoRIT(zkw);
-    master.assignmentManager.waitUntilNoRegionsInTransition(60000);
-    log("Assignment completed");
-
-    // Due to master.assignRegion(hri) could fail to assign a region to a specified RS
-    // therefore, we need make sure that regions are in the expected RS
-    verifyRegionLocation(hrs, enabledAndAssignedRegions);
-    verifyRegionLocation(hrs, disabledAndAssignedRegions);
-    verifyRegionLocation(hrsDead, enabledAndOnDeadRegions);
-    verifyRegionLocation(hrsDead, disabledAndOnDeadRegions);
-
-    assertTrue(" Didn't get enough regions of enabledTalbe on live rs.",
-      enabledAndAssignedRegions.size() >= 2);
-    assertTrue(" Didn't get enough regions of disalbedTable on live rs.",
-      disabledAndAssignedRegions.size() >= 2);
-    assertTrue(" Didn't get enough regions of enabledTalbe on dead rs.",
-      enabledAndOnDeadRegions.size() >= 2);
-    assertTrue(" Didn't get enough regions of disalbedTable on dead rs.",
-      disabledAndOnDeadRegions.size() >= 2);
-
-    // Stop the master
-    log("Aborting master");
-    cluster.abortMaster(0);
-    cluster.waitOnMaster(0);
-    log("Master has aborted");
-
-    /*
-     * Now, let's start mocking up some weird states as described in the method
-     * javadoc.
-     */
-
-    // Master is down, so is the meta. We need to assign it somewhere
-    // so that regions can be assigned during the mocking phase.
-    ZKAssign.createNodeOffline(
-      zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(),
-      hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
-
-    MetaTableLocator mtl = new MetaTableLocator();
-    while (true) {
-      ServerName sn = mtl.getMetaRegionLocation(zkw);
-      if (sn != null && sn.equals(hrs.getServerName())) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-
-    List<HRegionInfo> regionsThatShouldBeOnline = new ArrayList<HRegionInfo>();
-    List<HRegionInfo> regionsThatShouldBeOffline = new ArrayList<HRegionInfo>();
-
-    log("Beginning to mock scenarios");
-
-    // Disable the disabledTable in ZK
-    TableStateManager zktable = new ZKTableStateManager(zkw);
-    zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED);
-
-    assertTrue(" The enabled table should be identified on master fail over.",
-        zktable.isTableState(TableName.valueOf("enabledTable"),
-          ZooKeeperProtos.Table.State.ENABLED));
-
-    /*
-     * ZK = CLOSING
-     */
-
-    // Region of enabled table being closed on dead RS but not finished
-    HRegionInfo region = enabledAndOnDeadRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeClosing(zkw, region, deadServerName);
-    LOG.debug("\n\nRegion of enabled table was CLOSING on dead RS\n" +
-        region + "\n\n");
-
-    // Region of disabled table being closed on dead RS but not finished
-    region = disabledAndOnDeadRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    ZKAssign.createNodeClosing(zkw, region, deadServerName);
-    LOG.debug("\n\nRegion of disabled table was CLOSING on dead RS\n" +
-        region + "\n\n");
-
-    /*
-     * ZK = CLOSED
-     */
-
-    // Region of enabled on dead server gets closed but not ack'd by master
-    region = enabledAndOnDeadRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    int version = ZKAssign.createNodeClosing(zkw, region, deadServerName);
-    ZKAssign.transitionNodeClosed(zkw, region, deadServerName, version);
-    LOG.debug("\n\nRegion of enabled table was CLOSED on dead RS\n" +
-        region + "\n\n");
-
-    // Region of disabled on dead server gets closed but not ack'd by master
-    region = disabledAndOnDeadRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    version = ZKAssign.createNodeClosing(zkw, region, deadServerName);
-    ZKAssign.transitionNodeClosed(zkw, region, deadServerName, version);
-    LOG.debug("\n\nRegion of disabled table was CLOSED on dead RS\n" +
-        region + "\n\n");
-
-    /*
-     * ZK = OPENING
-     */
-
-    // RS was opening a region of enabled table then died
-    region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ZKAssign.transitionNodeOpening(zkw, region, deadServerName);
-    LOG.debug("\n\nRegion of enabled table was OPENING on dead RS\n" +
-        region + "\n\n");
-
-    // RS was opening a region of disabled table then died
-    region = disabledRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ZKAssign.transitionNodeOpening(zkw, region, deadServerName);
-    LOG.debug("\n\nRegion of disabled table was OPENING on dead RS\n" +
-        region + "\n\n");
-
-    /*
-     * ZK = OPENED
-     */
-
-    // Region of enabled table was opened on dead RS
-    region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
-      hrsDead.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-    LOG.debug("\n\nRegion of enabled table was OPENED on dead RS\n" +
-        region + "\n\n");
-
-    // Region of disabled table was opened on dead RS
-    region = disabledRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
-      hrsDead.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-    LOG.debug("\n\nRegion of disabled table was OPENED on dead RS\n" +
-        region + "\n\n");
-
-    /*
-     * ZK = NONE
-     */
-
-    // Region of enabled table was open at steady-state on dead RS
-    region = enabledRegions.remove(0);
-    regionsThatShouldBeOnline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
-      hrsDead.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        ZKAssign.deleteOpenedNode(zkw, region.getEncodedName(), rt.getServerName());
-        LOG.debug("DELETED " + rt);
-        break;
-      }
-      Thread.sleep(100);
-    }
-    LOG.debug("\n\nRegion of enabled table was open at steady-state on dead RS"
-        + "\n" + region + "\n\n");
-
-    // Region of disabled table was open at steady-state on dead RS
-    region = disabledRegions.remove(0);
-    regionsThatShouldBeOffline.add(region);
-    ZKAssign.createNodeOffline(zkw, region, deadServerName);
-    ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
-      hrsDead.getServerName(), region);
-    while (true) {
-      byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
-      RegionTransition rt = RegionTransition.parseFrom(bytes);
-      if (rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_OPENED)) {
-        ZKAssign.deleteOpenedNode(zkw, region.getEncodedName(), rt.getServerName());
-        break;
-      }
-      Thread.sleep(100);
-    }
-    LOG.debug("\n\nRegion of disabled table was open at steady-state on dead RS"
-      + "\n" + region + "\n\n");
-
-    /*
-     * DONE MOCKING
-     */
-
-    log("Done mocking data up in ZK");
-
-    // Kill the RS that had a hard death
-    log("Killing RS " + deadServerName);
-    hrsDead.abort("Killing for unit test");
-    log("RS " + deadServerName + " killed");
-
-    // Start up a new master.  Wait until regionserver is completely down
-    // before starting new master because of hbase-4511.
-    while (hrsDeadThread.isAlive()) {
-      Threads.sleep(10);
-    }
-    log("Starting up a new master");
-    master = cluster.startMaster().getMaster();
-    log("Waiting for master to be ready");
-    assertTrue(cluster.waitForActiveAndReadyMaster());
-    log("Master is ready");
-
-    // Wait until SSH processing completed for dead server.
-    while (master.getServerManager().areDeadServersInProgress()) {
-      Thread.sleep(10);
-    }
-
-    // Failover should be completed, now wait for no RIT
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-    log("No more RIT in ZK");
-    long now = System.currentTimeMillis();
-    long maxTime = 120000;
-    boolean done = master.assignmentManager.waitUntilNoRegionsInTransition(maxTime);
-    if (!done) {
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      LOG.info("rit=" + regionStates.getRegionsInTransition());
-    }
-    long elapsed = System.currentTimeMillis() - now;
-    assertTrue("Elapsed=" + elapsed + ", maxTime=" + maxTime + ", done=" + done,
-      elapsed < maxTime);
-    log("No more RIT in RIT map, doing final test verification");
-
-    // Grab all the regions that are online across RSs
-    Set<HRegionInfo> onlineRegions = new TreeSet<HRegionInfo>();
-    now = System.currentTimeMillis();
-    maxTime = 30000;
-    for (JVMClusterUtil.RegionServerThread rst :
-        cluster.getRegionServerThreads()) {
-      try {
-        HRegionServer rs = rst.getRegionServer();
-        while (!rs.getRegionsInTransitionInRS().isEmpty()) {
-          elapsed = System.currentTimeMillis() - now;
-          assertTrue("Test timed out in getting online regions", elapsed < maxTime);
-          if (rs.isAborted() || rs.isStopped()) {
-            // This region server is stopped, skip it.
-            break;
-          }
-          Thread.sleep(100);
-        }
-        onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()));
-      } catch (RegionServerStoppedException e) {
-        LOG.info("Got RegionServerStoppedException", e);
-      }
-    }
-
-    // Now, everything that should be online should be online
-    for (HRegionInfo hri : regionsThatShouldBeOnline) {
-      assertTrue("region=" + hri.getRegionNameAsString() + ", " + onlineRegions.toString(),
-        onlineRegions.contains(hri));
-    }
-
-    // Everything that should be offline should not be online
-    for (HRegionInfo hri : regionsThatShouldBeOffline) {
-      assertFalse(onlineRegions.contains(hri));
-    }
-
-    log("Done with verification, all passed, shutting down cluster");
-
-    // Done, shutdown the cluster
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Verify regions are on the expected region server
-   */
-  private void verifyRegionLocation(HRegionServer hrs, List<HRegionInfo> regions)
-      throws IOException {
-    List<HRegionInfo> tmpOnlineRegions =
-      ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
-    Iterator<HRegionInfo> itr = regions.iterator();
-    while (itr.hasNext()) {
-      HRegionInfo tmp = itr.next();
-      if (!tmpOnlineRegions.contains(tmp)) {
-        itr.remove();
-      }
-    }
-  }
-
   HRegion createRegion(final HRegionInfo  hri, final Path rootdir, final Configuration c,
       final HTableDescriptor htd)
   throws IOException {
@@ -970,121 +76,6 @@ public class TestMasterFailover {
     LOG.info("\n\n" + string + " \n\n");
   }
 
-  @Test (timeout=180000)
-  public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
-      throws Exception {
-    LOG.info("Starting testShouldCheckMasterFailOverWhenMETAIsInOpenedState");
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 2;
-
-    // Start the cluster
-    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();
-
-    // Find regionserver carrying meta.
-    HRegionServer regionServer = cluster.getMaster();
-    HRegion metaRegion = regionServer.getOnlineRegion(
-      HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
-
-    TEST_UTIL.shutdownMiniHBaseCluster();
-
-    // Create a ZKW to use in the test
-    ZooKeeperWatcher zkw =
-      HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
-          metaRegion, regionServer.getServerName());
-
-    LOG.info("Staring cluster for second time");
-    TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
-
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    while (!master.isInitialized()) {
-      Thread.sleep(100);
-    }
-    // Failover should be completed, now wait for no RIT
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-
-    zkw.close();
-    // Stop the cluster
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * This tests a RIT in offline state will get re-assigned after a master restart
-   */
-  @Test(timeout=240000)
-  public void testOfflineRegionReAssginedAfterMasterRestart() throws Exception {
-    final TableName table = TableName.valueOf("testOfflineRegionReAssginedAfterMasterRestart");
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 2;
-
-    // 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);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    log("Cluster started");
-
-    TEST_UTIL.createTable(table, Bytes.toBytes("family"));
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-    HRegionInfo hri = regionStates.getRegionsOfTable(table).get(0);
-    ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-    TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-
-    ServerName dstName = null;
-    for (ServerName tmpServer : master.serverManager.getOnlineServers().keySet()) {
-      if (!tmpServer.equals(serverName)) {
-        dstName = tmpServer;
-        break;
-      }
-    }
-    // find a different server
-    assertTrue(dstName != null);
-    // shutdown HBase cluster
-    TEST_UTIL.shutdownMiniHBaseCluster();
-    // create a RIT node in offline state
-    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
-    ZKAssign.createNodeOffline(zkw, hri, dstName);
-    Stat stat = new Stat();
-    byte[] data =
-        ZKAssign.getDataNoWatch(zkw, hri.getEncodedName(), stat);
-    assertTrue(data != null);
-    RegionTransition rt = RegionTransition.parseFrom(data);
-    assertTrue(rt.getEventType() == EventType.M_ZK_REGION_OFFLINE);
-
-    LOG.info(hri.getEncodedName() + " region is in offline state with source server=" + serverName
-        + " and dst server=" + dstName);
-
-    // start HBase cluster
-    TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
-
-    while (true) {
-      master = TEST_UTIL.getHBaseCluster().getMaster();
-      if (master != null && master.isInitialized()) {
-        ServerManager serverManager = master.getServerManager();
-        if (!serverManager.areDeadServersInProgress()) {
-          break;
-        }
-      }
-      Thread.sleep(200);
-    }
-
-    // verify the region is assigned
-    master = TEST_UTIL.getHBaseCluster().getMaster();
-    master.getAssignmentManager().waitForAssignment(hri);
-    regionStates = master.getAssignmentManager().getRegionStates();
-    RegionState newState = regionStates.getRegionState(hri);
-    assertTrue(newState.isOpened());
-  }
-
   /**
    * Simple test of master failover.
    * <p>
@@ -1205,7 +196,6 @@ public class TestMasterFailover {
 
     // 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);
@@ -1277,7 +267,7 @@ public class TestMasterFailover {
     log("Master is ready");
 
     // Wait till no region in transition any more
-    master.getAssignmentManager().waitUntilNoRegionsInTransition(60000);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     // Get new region states since master restarted
     regionStates = master.getAssignmentManager().getRegionStates();

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 00982b0..9f18f87 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -200,7 +199,7 @@ public class TestMasterNoCluster {
         // Fake a successful close.
         Mockito.doReturn(true).when(spy).
           sendRegionClose((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
-            Mockito.anyInt(), (ServerName)Mockito.any(), Mockito.anyBoolean());
+            (ServerName)Mockito.any());
         return spy;
       }
 
@@ -235,13 +234,8 @@ public class TestMasterNoCluster {
         request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
         master.getMasterRpcServices().regionServerReport(null, request.build());
       }
-      ZooKeeperWatcher zkw = master.getZooKeeper();
-      // Master should now come up.
+       // Master should now come up.
       while (!master.isInitialized()) {
-        // Fake meta is closed on rs0, try several times in case the event is lost
-        // due to race with HMaster#assignMeta
-        ZKAssign.transitionNodeClosed(zkw,
-          HRegionInfo.FIRST_META_REGIONINFO, sn0, -1);
         Threads.sleep(100);
       }
       assertTrue(master.isInitialized());

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
index 7ed455e..7ff8ca1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
@@ -37,9 +37,6 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -63,8 +60,6 @@ public class TestMasterRestartAfterDisablingTable {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     log("Waiting for active/ready master");
     cluster.waitForActiveAndReadyMaster();
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testmasterRestart", null);
-    HMaster master = cluster.getMaster();
 
     // Create a table with regions
     TableName table = TableName.valueOf("tableRestart");
@@ -75,7 +70,7 @@ public class TestMasterRestartAfterDisablingTable {
         NUM_REGIONS_TO_CREATE);
     numRegions += 1; // catalogs
     log("Waiting for no more RIT\n");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     log("Disabling table\n");
     TEST_UTIL.getHBaseAdmin().disableTable(table);
 
@@ -106,7 +101,7 @@ public class TestMasterRestartAfterDisablingTable {
     admin.enableTable(table);
     admin.close();
     log("Waiting for no more RIT\n");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     log("Verifying there are " + numRegions + " assigned on cluster\n");
     regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
     assertEquals("The assigned regions were not onlined after master"
@@ -122,11 +117,5 @@ public class TestMasterRestartAfterDisablingTable {
   private void log(String msg) {
     LOG.debug("\n\nTRR: " + msg + "\n");
   }
-
-  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
-      throws KeeperException, InterruptedException {
-    ZKAssign.blockUntilNoRIT(zkw);
-    master.assignmentManager.waitUntilNoRegionsInTransition(60000);
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index 2cae6fb..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/**
- * 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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MockServer;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-@Category(MediumTests.class)
-public class TestOpenedRegionHandler {
-
-  private static final Log LOG = LogFactory
-      .getLog(TestOpenedRegionHandler.class);
-
-  private HBaseTestingUtility TEST_UTIL;
-  private final int NUM_MASTERS = 1;
-  private final int NUM_RS = 1;
-  private Configuration conf;
-  private Configuration resetConf;
-  private ZooKeeperWatcher zkw;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = HBaseConfiguration.create();
-    conf.setBoolean("hbase.assignment.usezk", true);
-    TEST_UTIL = HBaseTestingUtility.createLocalHTU(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    // Stop the cluster
-    TEST_UTIL.shutdownMiniCluster();
-    TEST_UTIL = new HBaseTestingUtility(resetConf);
-  }
-
-  @Test
-  public void testOpenedRegionHandlerOnMasterRestart() throws Exception {
-    // Start the cluster
-    log("Starting cluster");
-    conf = HBaseConfiguration.create();
-    conf.setBoolean("hbase.assignment.usezk", true);
-    resetConf = conf;
-    TEST_UTIL = new HBaseTestingUtility(conf);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    String tableName = "testOpenedRegionHandlerOnMasterRestart";
-    MiniHBaseCluster cluster = createRegions(tableName);
-    abortMaster(cluster);
-
-    HRegionServer regionServer = cluster.getRegionServer(0);
-    HRegion region = getRegionBeingServed(cluster, regionServer);
-
-    // forcefully move a region to OPENED state in zk
-    // Create a ZKW to use in the test
-    zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
-        region, regionServer.getServerName());
-
-    // Start up a new master
-    log("Starting up a new master");
-    cluster.startMaster().getMaster();
-    log("Waiting for master to be ready");
-    cluster.waitForActiveAndReadyMaster();
-    log("Master is ready");
-
-    // Failover should be completed, now wait for no RIT
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-  }
-  @Test
-  public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
-      throws Exception {
-    HRegion region = null;
-    try {
-      int testIndex = 0;
-      TEST_UTIL.startMiniZKCluster();
-      final Server server = new MockServer(TEST_UTIL);
-      HTableDescriptor htd = new HTableDescriptor(
-          TableName.valueOf("testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches"));
-      HRegionInfo hri = new HRegionInfo(htd.getTableName(),
-          Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
-      region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
-      assertNotNull(region);
-      AssignmentManager am = Mockito.mock(AssignmentManager.class);
-      RegionStates rsm = Mockito.mock(RegionStates.class);
-      Mockito.doReturn(rsm).when(am).getRegionStates();
-      when(rsm.isRegionInTransition(hri)).thenReturn(false);
-      when(rsm.getRegionState(hri)).thenReturn(
-        new RegionState(region.getRegionInfo(), RegionState.State.OPEN,
-          System.currentTimeMillis(), server.getServerName()));
-      // create a node with OPENED state
-      zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
-          region, server.getServerName());
-      when(am.getTableStateManager()).thenReturn(new ZKTableStateManager(zkw));
-      Stat stat = new Stat();
-      String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo()
-          .getEncodedName());
-      ZKUtil.getDataAndWatch(zkw, nodeName, stat);
-
-      // use the version for the OpenedRegionHandler
-      BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
-      csm.initialize(server);
-      csm.start();
-
-      OpenRegionCoordination orc = csm.getOpenRegionCoordination();
-      ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
-        new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-      zkOrd.setServerName(server.getServerName());
-      zkOrd.setVersion(stat.getVersion());
-      OpenedRegionHandler handler = new OpenedRegionHandler(server, am, region
-          .getRegionInfo(), orc, zkOrd);
-      // Once again overwrite the same znode so that the version changes.
-      ZKAssign.transitionNode(zkw, region.getRegionInfo(), server
-          .getServerName(), EventType.RS_ZK_REGION_OPENED,
-          EventType.RS_ZK_REGION_OPENED, stat.getVersion());
-
-      // Should not invoke assignmentmanager.regionOnline. If it is 
-      // invoked as per current mocking it will throw null pointer exception.
-      boolean expectedException = false;
-      try {
-        handler.process();
-      } catch (Exception e) {
-        expectedException = true;
-      }
-      assertFalse("The process method should not throw any exception.",
-          expectedException);
-      List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
-          zkw.assignmentZNode);
-      String regionName = znodes.get(0);
-      assertEquals("The region should not be opened successfully.", regionName,
-          region.getRegionInfo().getEncodedName());
-    } finally {
-      HRegion.closeHRegion(region);
-      TEST_UTIL.shutdownMiniZKCluster();
-    }
-  }
-  private MiniHBaseCluster createRegions(String tableName)
-      throws InterruptedException, ZooKeeperConnectionException, IOException,
-      KeeperException {
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    log("Waiting for active/ready master");
-    cluster.waitForActiveAndReadyMaster();
-    zkw = new ZooKeeperWatcher(conf, "testOpenedRegionHandler", null);
-
-    // Create a table with regions
-    byte[] table = Bytes.toBytes(tableName);
-    byte[] family = Bytes.toBytes("family");
-    TEST_UTIL.createTable(table, family);
-
-    //wait till the regions are online
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-
-    return cluster;
-  }
-  private void abortMaster(MiniHBaseCluster cluster) {
-    // Stop the master
-    log("Aborting master");
-    cluster.abortMaster(0);
-    cluster.waitOnMaster(0);
-    log("Master has aborted");
-  }
-  private HRegion getRegionBeingServed(MiniHBaseCluster cluster,
-      HRegionServer regionServer) {
-    Collection<HRegion> onlineRegionsLocalContext = regionServer
-        .getOnlineRegionsLocalContext();
-    Iterator<HRegion> iterator = onlineRegionsLocalContext.iterator();
-    HRegion region = null;
-    while (iterator.hasNext()) {
-      region = iterator.next();
-      if (!region.getRegionInfo().isMetaTable()) {
-        break;
-      }
-    }
-    return region;
-  }
-  private void log(String msg) {
-    LOG.debug("\n\nTRR: " + msg + "\n");
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 c5ed9e9..7fc00b5 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
@@ -28,15 +28,18 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.MetaScanner;
-import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -46,8 +49,6 @@ public class TestRestartCluster {
   private static final Log LOG = LogFactory.getLog(TestRestartCluster.class);
   private HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  private static final byte[] TABLENAME = Bytes.toBytes("master_transitions");
-  private static final byte [][] FAMILIES = {Bytes.toBytes("a")};
   private static final byte [][] TABLES = {
       Bytes.toBytes("restartTableOne"),
       Bytes.toBytes("restartTableTwo"),
@@ -59,35 +60,6 @@ public class TestRestartCluster {
     UTIL.shutdownMiniCluster();
   }
 
-  @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);
-
-    // create the unassigned region, throw up a region opened state for META
-    String unassignedZNode = zooKeeper.assignmentZNode;
-    ZKUtil.createAndFailSilent(zooKeeper, unassignedZNode);
-
-    ServerName sn = ServerName.valueOf(HMaster.MASTER, 1, System.currentTimeMillis());
-
-    ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO, sn);
-
-    LOG.debug("Created UNASSIGNED zNode for ROOT and hbase:meta regions in state " +
-        EventType.M_ZK_REGION_OFFLINE);
-
-    // start the HB cluster
-    LOG.info("Starting HBase cluster...");
-    UTIL.startMiniCluster(2);
-
-    UTIL.createTable(TABLENAME, FAMILIES);
-    LOG.info("Created a table, waiting for table to be available...");
-    UTIL.waitTableAvailable(TABLENAME, 60*1000);
-
-    LOG.info("Master deleted unassigned region and started up successfully.");
-  }
-
   @Test (timeout=300000)
   public void testClusterRestart() throws Exception {
     UTIL.startMiniCluster(3);
@@ -153,8 +125,7 @@ public class TestRestartCluster {
     }
 
     HMaster master = UTIL.getMiniHBaseCluster().getMaster();
-    AssignmentManager am = master.getAssignmentManager();
-    am.waitUntilNoRegionsInTransition(120000);
+    UTIL.waitUntilNoRegionsInTransition(120000);
 
     // We don't have to use SnapshotOfRegionAssignmentFromMeta.
     // We use it here because AM used to use it to load all user region placements

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
index d2ab764..8bdafda 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
@@ -41,9 +41,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -72,9 +69,6 @@ public class  TestRollingRestart {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     log("Waiting for active/ready master");
     cluster.waitForActiveAndReadyMaster();
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testRollingRestart",
-        null);
-    HMaster master = cluster.getMaster();
 
     // Create a table with regions
     TableName table = TableName.valueOf("tableRestart");
@@ -85,11 +79,11 @@ public class  TestRollingRestart {
         NUM_REGIONS_TO_CREATE);
     numRegions += 1; // catalogs
     log("Waiting for no more RIT\n");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     log("Disabling table\n");
     TEST_UTIL.getHBaseAdmin().disableTable(table);
     log("Waiting for no more RIT\n");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
     log("Verifying only catalog and namespace regions are assigned\n");
     if (regions.size() != 2) {
@@ -99,7 +93,7 @@ public class  TestRollingRestart {
     log("Enabling table\n");
     TEST_UTIL.getHBaseAdmin().enableTable(table);
     log("Waiting for no more RIT\n");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     log("Verifying there are " + numRegions + " assigned on cluster\n");
     regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
     assertRegionsAssigned(cluster, regions);
@@ -112,7 +106,7 @@ public class  TestRollingRestart {
     restarted.waitForServerOnline();
     log("Additional RS is online");
     log("Waiting for no more RIT");
-    blockUntilNoRIT(zkw, master);
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
     log("Verifying there are " + numRegions + " assigned on cluster");
     assertRegionsAssigned(cluster, regions);
     assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
@@ -144,7 +138,6 @@ public class  TestRollingRestart {
     log("Restarting primary master\n\n");
     activeMaster = cluster.startMaster();
     cluster.waitForActiveAndReadyMaster();
-    master = activeMaster.getMaster();
 
     // Start backup master
     log("Restarting backup master\n\n");
@@ -168,7 +161,7 @@ public class  TestRollingRestart {
       log("Waiting for RS shutdown to be handled by master");
       waitForRSShutdownToStartAndFinish(activeMaster, serverName);
       log("RS shutdown done, waiting for no more RIT");
-      blockUntilNoRIT(zkw, master);
+      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       log("Verifying there are " + numRegions + " assigned on cluster");
       assertRegionsAssigned(cluster, regions);
       expectedNumRS--;
@@ -179,7 +172,7 @@ public class  TestRollingRestart {
       expectedNumRS++;
       log("Region server " + num + " is back online");
       log("Waiting for no more RIT");
-      blockUntilNoRIT(zkw, master);
+      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       log("Verifying there are " + numRegions + " assigned on cluster");
       assertRegionsAssigned(cluster, regions);
       assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
@@ -195,12 +188,6 @@ public class  TestRollingRestart {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
-  throws KeeperException, InterruptedException {
-    ZKAssign.blockUntilNoRIT(zkw);
-    master.assignmentManager.waitUntilNoRegionsInTransition(60000);
-  }
-
   private void waitForRSShutdownToStartAndFinish(MasterThread activeMaster,
       ServerName serverName) throws InterruptedException {
     ServerManager sm = activeMaster.getMaster().getServerManager();

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index e684d1e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
+++ /dev/null
@@ -1,302 +0,0 @@
-/**
- *
- * 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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.TableDescriptors;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.internal.util.reflection.Whitebox;
-
-/**
- * Test open and close of regions using zk.
- */
-@Category(MediumTests.class)
-public class TestZKBasedOpenCloseRegion {
-  private static final Log LOG = LogFactory.getLog(TestZKBasedOpenCloseRegion.class);
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final TableName TABLENAME =
-      TableName.valueOf("TestZKBasedOpenCloseRegion");
-  private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a"),
-    Bytes.toBytes("b"), Bytes.toBytes("c")};
-  private static int countOfRegions;
-
-  @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);
-    TEST_UTIL.createTable(TABLENAME, FAMILIES);
-    HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
-    countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
-    waitUntilAllRegionsAssigned();
-    addToEachStartKey(countOfRegions);
-    t.close();
-    TEST_UTIL.getHBaseCluster().getMaster().assignmentManager.initializeHandlerTrackers();
-  }
-
-  @AfterClass public static void afterAllTests() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before public void setup() throws IOException {
-    if (TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size() < 2) {
-      // Need at least two servers.
-      LOG.info("Started new server=" +
-        TEST_UTIL.getHBaseCluster().startRegionServer());
-
-    }
-    waitUntilAllRegionsAssigned();
-    waitOnRIT();
-  }
-
-  /**
-   * Test we reopen a region once closed.
-   * @throws Exception
-   */
-  @Test (timeout=300000) public void testReOpenRegion()
-  throws Exception {
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    LOG.info("Number of region servers = " +
-      cluster.getLiveRegionServerThreads().size());
-
-    int rsIdx = 0;
-    HRegionServer regionServer =
-      TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
-    HRegionInfo hri = getNonMetaRegion(
-      ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices()));
-    LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
-
-    LOG.info("Unassign " + hri.getRegionNameAsString());
-    cluster.getMaster().assignmentManager.unassign(hri);
-
-    while (!cluster.getMaster().assignmentManager.wasClosedHandlerCalled(hri)) {
-      Threads.sleep(100);
-    }
-
-    while (!cluster.getMaster().assignmentManager.wasOpenedHandlerCalled(hri)) {
-      Threads.sleep(100);
-    }
-
-    LOG.info("Done with testReOpenRegion");
-  }
-
-  private HRegionInfo getNonMetaRegion(final Collection<HRegionInfo> regions) {
-    HRegionInfo hri = null;
-    for (HRegionInfo i: regions) {
-      LOG.info(i.getRegionNameAsString());
-      if (!i.isMetaRegion()) {
-        hri = i;
-        break;
-      }
-    }
-    return hri;
-  }
-
-  /**
-   * This test shows how a region won't be able to be assigned to a RS
-   * if it's already "processing" it.
-   * @throws Exception
-   */
-  @Test
-  public void testRSAlreadyProcessingRegion() throws Exception {
-    LOG.info("starting testRSAlreadyProcessingRegion");
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-
-    HRegionServer hr0 =
-        cluster.getLiveRegionServerThreads().get(0).getRegionServer();
-    HRegionServer hr1 =
-        cluster.getLiveRegionServerThreads().get(1).getRegionServer();
-    HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr0.getRSRpcServices()));
-
-    // fake that hr1 is processing the region
-    hr1.getRegionsInTransitionInRS().putIfAbsent(hri.getEncodedNameAsBytes(), true);
-
-    // now ask the master to move the region to hr1, will fail
-    TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
-        Bytes.toBytes(hr1.getServerName().toString()));
-
-    // make sure the region came back
-    assertEquals(hr1.getOnlineRegion(hri.getEncodedNameAsBytes()), null);
-
-    // remove the block and reset the boolean
-    hr1.getRegionsInTransitionInRS().remove(hri.getEncodedNameAsBytes());
-
-    // now try moving a region when there is no region in transition.
-    hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1.getRSRpcServices()));
-
-    TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
-        Bytes.toBytes(hr0.getServerName().toString()));
-
-    while (!cluster.getMaster().assignmentManager.wasOpenedHandlerCalled(hri)) {
-      Threads.sleep(100);
-    }
-
-    // make sure the region has moved from the original RS
-    assertTrue(hr1.getOnlineRegion(hri.getEncodedNameAsBytes()) == null);
-
-  }
-
-  private void waitOnRIT() {
-    // Close worked but we are going to open the region elsewhere.  Before going on, make sure
-    // this completes.
-    while (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
-        getRegionStates().isRegionsInTransition()) {
-      LOG.info("Waiting on regions in transition: " +
-        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
-          getRegionStates().getRegionsInTransition());
-      Threads.sleep(10);
-    }
-  }
-
-  /**
-   * If region open fails with IOException in openRegion() while doing tableDescriptors.get()
-   * the region should not add into regionsInTransitionInRS map
-   * @throws Exception
-   */
-  @Test
-  public void testRegionOpenFailsDueToIOException() throws Exception {
-    HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("t"),
-        HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-    HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0);
-    TableDescriptors htd = Mockito.mock(TableDescriptors.class);
-    Object orizinalState = Whitebox.getInternalState(regionServer,"tableDescriptors");
-    Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
-    Mockito.doThrow(new IOException()).when(htd).get((TableName) Mockito.any());
-    try {
-      ProtobufUtil.openRegion(regionServer.getRSRpcServices(),
-        regionServer.getServerName(), REGIONINFO);
-      fail("It should throw IOException ");
-    } catch (IOException e) {
-    }
-    Whitebox.setInternalState(regionServer, "tableDescriptors", orizinalState);
-    assertFalse("Region should not be in RIT",
-        regionServer.getRegionsInTransitionInRS().containsKey(REGIONINFO.getEncodedNameAsBytes()));
-  }
-
-  private static void waitUntilAllRegionsAssigned()
-  throws IOException {
-    HTable meta = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
-    while (true) {
-      int rows = 0;
-      Scan scan = new Scan();
-      scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
-      ResultScanner s = meta.getScanner(scan);
-      for (Result r = null; (r = s.next()) != null;) {
-        byte [] b =
-          r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
-        if (b == null || b.length <= 0) {
-          break;
-        }
-        rows++;
-      }
-      s.close();
-      // If I get to here and all rows have a Server, then all have been assigned.
-      if (rows >= countOfRegions) {
-        break;
-      }
-      LOG.info("Found=" + rows);
-      Threads.sleep(1000);
-    }
-    meta.close();
-  }
-
-  /*
-   * Add to each of the regions in hbase:meta a value.  Key is the startrow of the
-   * region (except its 'aaa' for first region).  Actual value is the row name.
-   * @param expected
-   * @return
-   * @throws IOException
-   */
-  private static int addToEachStartKey(final int expected) throws IOException {
-    HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
-    HTable meta = new HTable(TEST_UTIL.getConfiguration(),
-        TableName.META_TABLE_NAME);
-    int rows = 0;
-    Scan scan = new Scan();
-    scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    ResultScanner s = meta.getScanner(scan);
-    for (Result r = null; (r = s.next()) != null;) {
-      HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
-      if (hri == null) break;
-      if(!hri.getTable().equals(TABLENAME)) {
-        continue;
-      }
-      // If start key, add 'aaa'.
-      byte [] row = getStartKey(hri);
-      Put p = new Put(row);
-      p.setDurability(Durability.SKIP_WAL);
-      p.add(getTestFamily(), getTestQualifier(), row);
-      t.put(p);
-      rows++;
-    }
-    s.close();
-    Assert.assertEquals(expected, rows);
-    t.close();
-    meta.close();
-    return rows;
-  }
-
-  private static byte [] getStartKey(final HRegionInfo hri) {
-    return Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey())?
-        Bytes.toBytes("aaa"): hri.getStartKey();
-  }
-
-  private static byte [] getTestFamily() {
-    return FAMILIES[0];
-  }
-
-  private static byte [] getTestQualifier() {
-    return getTestFamily();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index 83d33c5..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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();
-  }
-}


[07/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 4803227..c234767 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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -36,7 +35,6 @@ import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -55,24 +53,16 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.TableStateManager;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
-import org.apache.hadoop.hbase.coordination.RegionMergeCoordination;
-import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination.SplitTransactionDetails;
-import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
-import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -82,10 +72,8 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
-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.RegionStateTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
@@ -94,42 +82,25 @@ import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 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;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.util.Triple;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
-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.collect.LinkedHashMultimap;
 
 /**
  * Manages and performs region assignment.
- * <p>
- * Monitors ZooKeeper for events related to regions in transition.
- * <p>
- * Handles existing regions in transition during master failover.
+ * Related communications with regionserver are all done over RPC.
  */
 @InterfaceAudience.Private
-public class AssignmentManager extends ZooKeeperListener {
+public class AssignmentManager {
   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
 
-  public static final ServerName HBCK_CODE_SERVERNAME = ServerName.valueOf(HConstants.HBCK_CODE_NAME,
-      -1, -1L);
-
   static final String ALREADY_IN_TRANSITION_WAITTIME
     = "hbase.assignment.already.intransition.waittime";
   static final int DEFAULT_ALREADY_IN_TRANSITION_WAITTIME = 60000; // 1 minute
@@ -187,21 +158,9 @@ public class AssignmentManager extends ZooKeeperListener {
 
   private final ExecutorService executorService;
 
-  // For unit tests, keep track of calls to ClosedRegionHandler
-  private Map<HRegionInfo, AtomicBoolean> closedRegionHandlerCalled = null;
-
-  // For unit tests, keep track of calls to OpenedRegionHandler
-  private Map<HRegionInfo, AtomicBoolean> openedRegionHandlerCalled = null;
-
-  //Thread pool executor service for timeout monitor
+  // Thread pool executor service. TODO, consolidate with executorService?
   private java.util.concurrent.ExecutorService threadPoolExecutorService;
 
-  // A bunch of ZK events workers. Each is a single thread executor service
-  private final java.util.concurrent.ExecutorService zkEventWorkers;
-
-  private List<EventType> ignoreStatesRSOffline = Arrays.asList(
-      EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED);
-
   private final RegionStates regionStates;
 
   // The threshold to use bulk assigning. Using bulk assignment
@@ -236,9 +195,6 @@ 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;
@@ -261,15 +217,14 @@ public class AssignmentManager extends ZooKeeperListener {
    * @param service Executor service
    * @param metricsMaster metrics manager
    * @param tableLockManager TableLock manager
-   * @throws KeeperException
+   * @throws CoordinatedStateException
    * @throws IOException
    */
   public AssignmentManager(Server server, ServerManager serverManager,
       final LoadBalancer balancer,
       final ExecutorService service, MetricsMaster metricsMaster,
-      final TableLockManager tableLockManager) throws KeeperException,
-        IOException, CoordinatedStateException {
-    super(server.getZooKeeper());
+      final TableLockManager tableLockManager)
+          throws IOException, CoordinatedStateException {
     this.server = server;
     this.serverManager = serverManager;
     this.executorService = service;
@@ -307,14 +262,8 @@ public class AssignmentManager extends ZooKeeperListener {
     this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
     this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
 
-    int workers = conf.getInt("hbase.assignment.zkevent.workers", 20);
-    ThreadFactory threadFactory = Threads.newDaemonThreadFactory("AM.ZK.Worker");
-    zkEventWorkers = Threads.getBoundedCachedThreadPool(workers, 60L,
-            TimeUnit.SECONDS, threadFactory);
-    this.tableLockManager = tableLockManager;
-
     this.metricsAssignmentManager = new MetricsAssignmentManager();
-    useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
+    this.tableLockManager = tableLockManager;
   }
 
   /**
@@ -406,9 +355,9 @@ public class AssignmentManager extends ZooKeeperListener {
    */
   public Pair<Integer, Integer> getReopenStatus(TableName tableName)
       throws IOException {
-    List <HRegionInfo> hris =
-      MetaTableAccessor.getTableRegions(this.watcher, this.server.getShortCircuitConnection(),
-        tableName, true);
+    List <HRegionInfo> hris = MetaTableAccessor.getTableRegions(
+      this.server.getZooKeeper(), this.server.getShortCircuitConnection(),
+      tableName, true);
     Integer pending = 0;
     for (HRegionInfo hri : hris) {
       String name = hri.getEncodedName();
@@ -476,10 +425,6 @@ public class AssignmentManager extends ZooKeeperListener {
     // previous master process.
     boolean failover = processDeadServersAndRegionsInTransition(deadServers);
 
-    if (!useZKForAssignment) {
-      // Not use ZK for assignment any more, remove the ZNode
-      ZKUtil.deleteNodeRecursively(watcher, watcher.assignmentZNode);
-    }
     recoverTableInDisablingState();
     recoverTableInEnablingState();
     LOG.info("Joined the cluster in " + (System.currentTimeMillis()
@@ -493,22 +438,12 @@ public class AssignmentManager extends ZooKeeperListener {
    * startup, will assign all user regions.
    * @param deadServers
    *          Map of dead servers and their regions. Can be null.
-   * @throws KeeperException
    * @throws IOException
    * @throws InterruptedException
+   * @throws CoordinatedStateException
    */
-  boolean processDeadServersAndRegionsInTransition(
-      final Set<ServerName> deadServers) throws KeeperException,
-        IOException, InterruptedException, CoordinatedStateException {
-    List<String> nodes = ZKUtil.listChildrenNoWatch(watcher,
-      watcher.assignmentZNode);
-
-    if (useZKForAssignment && nodes == null) {
-      String errorMessage = "Failed to get the children from ZK";
-      server.abort(errorMessage, new IOException(errorMessage));
-      return true; // Doesn't matter in this case
-    }
-
+  boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
+      throws IOException, InterruptedException, CoordinatedStateException {
     boolean failover = !serverManager.getDeadServers().isEmpty();
     if (failover) {
       // This may not be a failover actually, especially if meta is on this master.
@@ -517,36 +452,28 @@ 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;
         }
       }
-    }
-    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;
+      if (!failover) {
+        // 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()) {
+          for (RegionState regionState: regionsInTransition.values()) {
+            if (!regionState.getRegion().isMetaRegion()
+                && onlineServers.contains(regionState.getServerName())) {
+              LOG.debug("Found " + regionState + " in RITs");
+              failover = true;
+              break;
+            }
           }
         }
       }
@@ -596,19 +523,8 @@ public class AssignmentManager extends ZooKeeperListener {
     // 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);
-    }
-
-    if (!failover && useZKForAssignment) {
-      // Cleanup any existing ZK nodes and start watching
-      ZKAssign.deleteAllNodes(watcher);
-      ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
-        this.watcher.assignmentZNode);
+      processDeadServers(deadServers);
     }
 
     // Now we can safely claim failover cleanup completed and enable
@@ -632,254 +548,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * If region is up in zk in transition, then do fixup and block and wait until
-   * the region is assigned and out of transition.  Used on startup for
-   * catalog regions.
-   * @param hri Region to look for.
-   * @return True if we processed a region in transition else false if region
-   * was not up in zk in transition.
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws IOException
-   */
-  boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
-      throws InterruptedException, KeeperException, IOException {
-    String encodedRegionName = hri.getEncodedName();
-    if (!processRegionInTransition(encodedRegionName, hri)) {
-      return false; // The region is not in transition
-    }
-    LOG.debug("Waiting on " + HRegionInfo.prettyPrint(encodedRegionName));
-    while (!this.server.isStopped() &&
-        this.regionStates.isRegionInTransition(encodedRegionName)) {
-      RegionState state = this.regionStates.getRegionTransitionState(encodedRegionName);
-      if (state == null || !serverManager.isServerOnline(state.getServerName())) {
-        // The region is not in transition, or not in transition on an online
-        // server. Doesn't help to block here any more. Caller need to
-        // verify the region is actually assigned.
-        break;
-      }
-      this.regionStates.waitForUpdate(100);
-    }
-    return true;
-  }
-
-  /**
-   * Process failover of new master for region <code>encodedRegionName</code>
-   * up in zookeeper.
-   * @param encodedRegionName Region to process failover for.
-   * @param regionInfo If null we'll go get it from meta table.
-   * @return True if we processed <code>regionInfo</code> as a RIT.
-   * @throws KeeperException
-   * @throws IOException
-   */
-  boolean processRegionInTransition(final String encodedRegionName,
-      final HRegionInfo regionInfo) throws KeeperException, IOException {
-    // We need a lock here to ensure that we will not put the same region twice
-    // It has no reason to be a lock shared with the other operations.
-    // We can do the lock on the region only, instead of a global lock: what we want to ensure
-    // is that we don't have two threads working on the same region.
-    Lock lock = locker.acquireLock(encodedRegionName);
-    try {
-      Stat stat = new Stat();
-      byte [] data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat);
-      if (data == null) return false;
-      RegionTransition rt;
-      try {
-        rt = RegionTransition.parseFrom(data);
-      } catch (DeserializationException e) {
-        LOG.warn("Failed parse znode data", e);
-        return false;
-      }
-      HRegionInfo hri = regionInfo;
-      if (hri == null) {
-        // The region info is not passed in. We will try to find the region
-        // from region states map/meta based on the encoded region name. But we
-        // may not be able to find it. This is valid for online merge that
-        // the region may have not been created if the merge is not completed.
-        // Therefore, it is not in meta at master recovery time.
-        hri = regionStates.getRegionInfo(rt.getRegionName());
-        EventType et = rt.getEventType();
-        if (hri == null && et != EventType.RS_ZK_REGION_MERGING
-            && et != EventType.RS_ZK_REQUEST_REGION_MERGE) {
-          LOG.warn("Couldn't find the region in recovering " + rt);
-          return false;
-        }
-      }
-
-      // TODO: This code is tied to ZK anyway, so for now leaving it as is,
-      // will refactor when whole region assignment will be abstracted from ZK
-      BaseCoordinatedStateManager cp =
-        (BaseCoordinatedStateManager) this.server.getCoordinatedStateManager();
-      OpenRegionCoordination openRegionCoordination = cp.getOpenRegionCoordination();
-
-      ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
-        new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-      zkOrd.setVersion(stat.getVersion());
-      zkOrd.setServerName(cp.getServer().getServerName());
-
-      return processRegionsInTransition(
-        rt, hri, openRegionCoordination, zkOrd);
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  /**
-   * This call is invoked only (1) master assign meta;
-   * (2) during failover mode startup, zk assignment node processing.
-   * The locker is set in the caller. It returns true if the region
-   * is in transition for sure, false otherwise.
-   *
-   * It should be private but it is used by some test too.
-   */
-  boolean processRegionsInTransition(
-      final RegionTransition rt, final HRegionInfo regionInfo,
-      OpenRegionCoordination coordination,
-      final OpenRegionCoordination.OpenRegionDetails ord) throws KeeperException {
-    EventType et = rt.getEventType();
-    // Get ServerName.  Could not be null.
-    final ServerName sn = rt.getServerName();
-    final byte[] regionName = rt.getRegionName();
-    final String encodedName = HRegionInfo.encodeRegionName(regionName);
-    final String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
-    LOG.info("Processing " + prettyPrintedRegionName + " in state: " + et);
-
-    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));
-      // Just return
-      return true;
-    }
-    if (!serverManager.isServerOnline(sn)) {
-      // It was transitioning on a dead server, so it's closed now.
-      // Force to OFFLINE and put it in transition, but not assign it
-      // since log splitting for the dead server is not done yet.
-      LOG.debug("RIT " + encodedName + " in state=" + rt.getEventType() +
-        " was on deadserver; forcing offline");
-      if (regionStates.isRegionOnline(regionInfo)) {
-        // Meta could still show the region is assigned to the previous
-        // server. If that server is online, when we reload the meta, the
-        // region is put back to online, we need to offline it.
-        regionStates.regionOffline(regionInfo);
-        sendRegionClosedNotification(regionInfo);
-      }
-      // Put it back in transition so that SSH can re-assign it
-      regionStates.updateRegionState(regionInfo, State.OFFLINE, sn);
-
-      if (regionInfo.isMetaRegion()) {
-        // If it's meta region, reset the meta location.
-        // So that master knows the right meta region server.
-        MetaTableLocator.setMetaLocation(watcher, sn);
-      } else {
-        // No matter the previous server is online or offline,
-        // we need to reset the last region server of the region.
-        regionStates.setLastRegionServerOfRegion(sn, encodedName);
-        // Make sure we know the server is dead.
-        if (!serverManager.isServerDead(sn)) {
-          serverManager.expireServer(sn);
-        }
-      }
-      return false;
-    }
-    switch (et) {
-      case M_ZK_REGION_CLOSING:
-        // Insert into RIT & resend the query to the region server: may be the previous master
-        // died before sending the query the first time.
-        final RegionState rsClosing = regionStates.updateRegionState(rt, State.CLOSING);
-        this.executorService.submit(
-          new EventHandler(server, EventType.M_MASTER_RECOVERY) {
-            @Override
-            public void process() throws IOException {
-              ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
-              try {
-                final int expectedVersion = ((ZkOpenRegionCoordination.ZkOpenRegionDetails) ord)
-                  .getVersion();
-                unassign(regionInfo, rsClosing, expectedVersion, null, useZKForAssignment, null);
-                if (regionStates.isRegionOffline(regionInfo)) {
-                  assign(regionInfo, true);
-                }
-              } finally {
-                lock.unlock();
-              }
-            }
-          });
-        break;
-
-      case RS_ZK_REGION_CLOSED:
-      case RS_ZK_REGION_FAILED_OPEN:
-        // Region is closed, insert into RIT and handle it
-        regionStates.updateRegionState(regionInfo, State.CLOSED, sn);
-        if (!replicasToClose.contains(regionInfo)) {
-          invokeAssign(regionInfo);
-        } else {
-          offlineDisabledRegion(regionInfo);
-        }
-        break;
-
-      case M_ZK_REGION_OFFLINE:
-        // Insert in RIT and resend to the regionserver
-        regionStates.updateRegionState(rt, State.PENDING_OPEN);
-        final RegionState rsOffline = regionStates.getRegionState(regionInfo);
-        this.executorService.submit(
-          new EventHandler(server, EventType.M_MASTER_RECOVERY) {
-            @Override
-            public void process() throws IOException {
-              ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
-              try {
-                RegionPlan plan = new RegionPlan(regionInfo, null, sn);
-                addPlan(encodedName, plan);
-                assign(rsOffline, false, false);
-              } finally {
-                lock.unlock();
-              }
-            }
-          });
-        break;
-
-      case RS_ZK_REGION_OPENING:
-        regionStates.updateRegionState(rt, State.OPENING);
-        break;
-
-      case RS_ZK_REGION_OPENED:
-        // Region is opened, insert into RIT and handle it
-        // This could be done asynchronously, we would need then to acquire the lock in the
-        //  handler.
-        regionStates.updateRegionState(rt, State.OPEN);
-        new OpenedRegionHandler(server, this, regionInfo, coordination, ord).process();
-        break;
-      case RS_ZK_REQUEST_REGION_SPLIT:
-      case RS_ZK_REGION_SPLITTING:
-      case RS_ZK_REGION_SPLIT:
-        // Splitting region should be online. We could have skipped it during
-        // user region rebuilding since we may consider the split is completed.
-        // Put it in SPLITTING state to avoid complications.
-        regionStates.regionOnline(regionInfo, sn);
-        regionStates.updateRegionState(rt, State.SPLITTING);
-        if (!handleRegionSplitting(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
-          deleteSplittingNode(encodedName, sn);
-        }
-        break;
-      case RS_ZK_REQUEST_REGION_MERGE:
-      case RS_ZK_REGION_MERGING:
-      case RS_ZK_REGION_MERGED:
-        if (!handleRegionMerging(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
-          deleteMergingNode(encodedName, sn);
-        }
-        break;
-      default:
-        throw new IllegalStateException("Received region in state:" + et + " is not valid.");
-    }
-    LOG.info("Processed region " + prettyPrintedRegionName + " in state "
-      + et + ", on " + (serverManager.isServerOnline(sn) ? "" : "dead ")
-      + "server: " + sn);
-    return true;
-  }
-
-  /**
    * When a region is closed, it should be removed from the regionsToReopen
    * @param hri HRegionInfo of the region which was closed
    */
@@ -889,247 +557,6 @@ public class AssignmentManager extends ZooKeeperListener {
     }
   }
 
-  /**
-   * Handles various states an unassigned node can be in.
-   * <p>
-   * Method is called when a state change is suspected for an unassigned node.
-   * <p>
-   * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
-   * yet).
-   * @param rt region transition
-   * @param coordination coordination for opening region
-   * @param ord details about opening region
-   */
-  void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
-                    OpenRegionCoordination.OpenRegionDetails ord) {
-    if (rt == null) {
-      LOG.warn("Unexpected NULL input for RegionTransition rt");
-      return;
-    }
-    final ServerName sn = rt.getServerName();
-    // Check if this is a special HBCK transition
-    if (sn.equals(HBCK_CODE_SERVERNAME)) {
-      handleHBCK(rt);
-      return;
-    }
-    final long createTime = rt.getCreateTime();
-    final byte[] regionName = rt.getRegionName();
-    String encodedName = HRegionInfo.encodeRegionName(regionName);
-    String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
-    // Verify this is a known server
-    if (!serverManager.isServerOnline(sn)
-      && !ignoreStatesRSOffline.contains(rt.getEventType())) {
-      LOG.warn("Attempted to handle region transition for server but " +
-        "it is not online: " + prettyPrintedRegionName + ", " + rt);
-      return;
-    }
-
-    RegionState regionState =
-      regionStates.getRegionState(encodedName);
-    long startTime = System.currentTimeMillis();
-    if (LOG.isDebugEnabled()) {
-      boolean lateEvent = createTime < (startTime - 15000);
-      LOG.debug("Handling " + rt.getEventType() +
-        ", server=" + sn + ", region=" +
-        (prettyPrintedRegionName == null ? "null" : prettyPrintedRegionName) +
-        (lateEvent ? ", which is more than 15 seconds late" : "") +
-        ", current_state=" + regionState);
-    }
-    // We don't do anything for this event,
-    // so separate it out, no need to lock/unlock anything
-    if (rt.getEventType() == EventType.M_ZK_REGION_OFFLINE) {
-      return;
-    }
-
-    // We need a lock on the region as we could update it
-    Lock lock = locker.acquireLock(encodedName);
-    try {
-      RegionState latestState =
-        regionStates.getRegionState(encodedName);
-      if ((regionState == null && latestState != null)
-          || (regionState != null && latestState == null)
-          || (regionState != null && latestState != null
-            && latestState.getState() != regionState.getState())) {
-        LOG.warn("Region state changed from " + regionState + " to "
-          + latestState + ", while acquiring lock");
-      }
-      long waitedTime = System.currentTimeMillis() - startTime;
-      if (waitedTime > 5000) {
-        LOG.warn("Took " + waitedTime + "ms to acquire the lock");
-      }
-      regionState = latestState;
-      switch (rt.getEventType()) {
-      case RS_ZK_REQUEST_REGION_SPLIT:
-      case RS_ZK_REGION_SPLITTING:
-      case RS_ZK_REGION_SPLIT:
-        if (!handleRegionSplitting(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
-          deleteSplittingNode(encodedName, sn);
-        }
-        break;
-
-      case RS_ZK_REQUEST_REGION_MERGE:
-      case RS_ZK_REGION_MERGING:
-      case RS_ZK_REGION_MERGED:
-        // Merged region is a new region, we can't find it in the region states now.
-        // However, the two merging regions are not new. They should be in state for merging.
-        if (!handleRegionMerging(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
-          deleteMergingNode(encodedName, sn);
-        }
-        break;
-
-      case M_ZK_REGION_CLOSING:
-        // Should see CLOSING after we have asked it to CLOSE or additional
-        // times after already being in state of CLOSING
-        if (regionState == null
-            || !regionState.isPendingCloseOrClosingOnServer(sn)) {
-          LOG.warn("Received CLOSING for " + prettyPrintedRegionName
-            + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
-            + regionStates.getRegionState(encodedName));
-          return;
-        }
-        // Transition to CLOSING (or update stamp if already CLOSING)
-        regionStates.updateRegionState(rt, State.CLOSING);
-        break;
-
-      case RS_ZK_REGION_CLOSED:
-        // Should see CLOSED after CLOSING but possible after PENDING_CLOSE
-        if (regionState == null
-            || !regionState.isPendingCloseOrClosingOnServer(sn)) {
-          LOG.warn("Received CLOSED for " + prettyPrintedRegionName
-            + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
-            + regionStates.getRegionState(encodedName));
-          return;
-        }
-        // Handle CLOSED by assigning elsewhere or stopping if a disable
-        // If we got here all is good.  Need to update RegionState -- else
-        // what follows will fail because not in expected state.
-        new ClosedRegionHandler(server, this, regionState.getRegion()).process();
-        updateClosedRegionHandlerTracker(regionState.getRegion());
-        break;
-
-        case RS_ZK_REGION_FAILED_OPEN:
-          if (regionState == null
-              || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
-            LOG.warn("Received FAILED_OPEN for " + prettyPrintedRegionName
-              + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
-              + regionStates.getRegionState(encodedName));
-            return;
-          }
-          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(rt, 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 = regionStates.updateRegionState(rt, 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 regionplan. (HBASE-5546)
-              try {
-                getRegionPlan(regionState.getRegion(), sn, true);
-                new ClosedRegionHandler(server, this, regionState.getRegion()).process();
-              } catch (HBaseIOException e) {
-                LOG.warn("Failed to get region plan", e);
-              }
-            }
-          }
-          break;
-
-        case RS_ZK_REGION_OPENING:
-          // Should see OPENING after we have asked it to OPEN or additional
-          // times after already being in state of OPENING
-          if (regionState == null
-              || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
-            LOG.warn("Received OPENING for " + prettyPrintedRegionName
-              + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
-              + regionStates.getRegionState(encodedName));
-            return;
-          }
-          // Transition to OPENING (or update stamp if already OPENING)
-          regionStates.updateRegionState(rt, State.OPENING);
-          break;
-
-        case RS_ZK_REGION_OPENED:
-          // Should see OPENED after OPENING but possible after PENDING_OPEN.
-          if (regionState == null
-              || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
-            LOG.warn("Received OPENED for " + prettyPrintedRegionName
-              + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
-              + regionStates.getRegionState(encodedName));
-
-            if (regionState != null) {
-              // Close it without updating the internal region states,
-              // so as not to create double assignments in unlucky scenarios
-              // mentioned in OpenRegionHandler#process
-              unassign(regionState.getRegion(), null, -1, null, false, sn);
-            }
-            return;
-          }
-          // Handle OPENED by removing from transition and deleted zk node
-          regionState = regionStates.updateRegionState(rt, State.OPEN);
-          if (regionState != null) {
-            failedOpenTracker.remove(encodedName); // reset the count, if any
-            new OpenedRegionHandler(
-              server, this, regionState.getRegion(), coordination, ord).process();
-            updateOpenedRegionHandlerTracker(regionState.getRegion());
-          }
-          break;
-
-        default:
-          throw new IllegalStateException("Received event is not valid.");
-      }
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  //For unit tests only
-  boolean wasClosedHandlerCalled(HRegionInfo hri) {
-    AtomicBoolean b = closedRegionHandlerCalled.get(hri);
-    //compareAndSet to be sure that unit tests don't see stale values. Means,
-    //we will return true exactly once unless the handler code resets to true
-    //this value.
-    return b == null ? false : b.compareAndSet(true, false);
-  }
-
-  //For unit tests only
-  boolean wasOpenedHandlerCalled(HRegionInfo hri) {
-    AtomicBoolean b = openedRegionHandlerCalled.get(hri);
-    //compareAndSet to be sure that unit tests don't see stale values. Means,
-    //we will return true exactly once unless the handler code resets to true
-    //this value.
-    return b == null ? false : b.compareAndSet(true, false);
-  }
-
-  //For unit tests only
-  void initializeHandlerTrackers() {
-    closedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
-    openedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
-  }
-
-  void updateClosedRegionHandlerTracker(HRegionInfo hri) {
-    if (closedRegionHandlerCalled != null) { //only for unit tests this is true
-      closedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
-    }
-  }
-
-  void updateOpenedRegionHandlerTracker(HRegionInfo hri) {
-    if (openedRegionHandlerCalled != null) { //only for unit tests this is true
-      openedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
-    }
-  }
-
   // TODO: processFavoredNodes might throw an exception, for e.g., if the
   // meta could not be contacted/updated. We need to see how seriously to treat
   // this problem as. Should we fail the current assignment. We should be able
@@ -1150,264 +577,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * Handle a ZK unassigned node transition triggered by HBCK repair tool.
-   * <p>
-   * This is handled in a separate code path because it breaks the normal rules.
-   * @param rt
-   */
-  @SuppressWarnings("deprecation")
-  private void handleHBCK(RegionTransition rt) {
-    String encodedName = HRegionInfo.encodeRegionName(rt.getRegionName());
-    LOG.info("Handling HBCK triggered transition=" + rt.getEventType() +
-      ", server=" + rt.getServerName() + ", region=" +
-      HRegionInfo.prettyPrint(encodedName));
-    RegionState regionState = regionStates.getRegionTransitionState(encodedName);
-    switch (rt.getEventType()) {
-      case M_ZK_REGION_OFFLINE:
-        HRegionInfo regionInfo;
-        if (regionState != null) {
-          regionInfo = regionState.getRegion();
-        } else {
-          try {
-            byte [] name = rt.getRegionName();
-            Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
-              this.server.getShortCircuitConnection(), name);
-            regionInfo = p.getFirst();
-          } catch (IOException e) {
-            LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
-            return;
-          }
-        }
-        LOG.info("HBCK repair is triggering assignment of region=" +
-            regionInfo.getRegionNameAsString());
-        // trigger assign, node is already in OFFLINE so don't need to update ZK
-        assign(regionInfo, false);
-        break;
-
-      default:
-        LOG.warn("Received unexpected region state from HBCK: " + rt.toString());
-        break;
-    }
-
-  }
-
-  // ZooKeeper events
-
-  /**
-   * New unassigned node has been created.
-   *
-   * <p>This happens when an RS begins the OPENING or CLOSING of a region by
-   * creating an unassigned node.
-   *
-   * <p>When this happens we must:
-   * <ol>
-   *   <li>Watch the node for further events</li>
-   *   <li>Read and handle the state in the node</li>
-   * </ol>
-   */
-  @Override
-  public void nodeCreated(String path) {
-    handleAssignmentEvent(path);
-  }
-
-  /**
-   * Existing unassigned node has had data changed.
-   *
-   * <p>This happens when an RS transitions from OFFLINE to OPENING, or between
-   * OPENING/OPENED and CLOSING/CLOSED.
-   *
-   * <p>When this happens we must:
-   * <ol>
-   *   <li>Watch the node for further events</li>
-   *   <li>Read and handle the state in the node</li>
-   * </ol>
-   */
-  @Override
-  public void nodeDataChanged(String path) {
-    handleAssignmentEvent(path);
-  }
-
-
-  // We  don't want to have two events on the same region managed simultaneously.
-  // For this reason, we need to wait if an event on the same region is currently in progress.
-  // So we track the region names of the events in progress, and we keep a waiting list.
-  private final Set<String> regionsInProgress = new HashSet<String>();
-  // In a LinkedHashMultimap, the put order is kept when we retrieve the collection back. We need
-  //  this as we want the events to be managed in the same order as we received them.
-  private final LinkedHashMultimap <String, RegionRunnable>
-      zkEventWorkerWaitingList = LinkedHashMultimap.create();
-
-  /**
-   * A specific runnable that works only on a region.
-   */
-  private interface RegionRunnable extends Runnable{
-    /**
-     * @return - the name of the region it works on.
-     */
-    String getRegionName();
-  }
-
-  /**
-   * Submit a task, ensuring that there is only one task at a time that working on a given region.
-   * Order is respected.
-   */
-  protected void zkEventWorkersSubmit(final RegionRunnable regRunnable) {
-
-    synchronized (regionsInProgress) {
-      // If we're there is already a task with this region, we add it to the
-      //  waiting list and return.
-      if (regionsInProgress.contains(regRunnable.getRegionName())) {
-        synchronized (zkEventWorkerWaitingList){
-          zkEventWorkerWaitingList.put(regRunnable.getRegionName(), regRunnable);
-        }
-        return;
-      }
-
-      // No event in progress on this region => we can submit a new task immediately.
-      regionsInProgress.add(regRunnable.getRegionName());
-      zkEventWorkers.submit(new Runnable() {
-        @Override
-        public void run() {
-          try {
-            regRunnable.run();
-          } finally {
-            // now that we have finished, let's see if there is an event for the same region in the
-            //  waiting list. If it's the case, we can now submit it to the pool.
-            synchronized (regionsInProgress) {
-              regionsInProgress.remove(regRunnable.getRegionName());
-              synchronized (zkEventWorkerWaitingList) {
-                java.util.Set<RegionRunnable> waiting = zkEventWorkerWaitingList.get(
-                    regRunnable.getRegionName());
-                if (!waiting.isEmpty()) {
-                  // We want the first object only. The only way to get it is through an iterator.
-                  RegionRunnable toSubmit = waiting.iterator().next();
-                  zkEventWorkerWaitingList.remove(toSubmit.getRegionName(), toSubmit);
-                  zkEventWorkersSubmit(toSubmit);
-                }
-              }
-            }
-          }
-        }
-      });
-    }
-  }
-
-  @Override
-  public void nodeDeleted(final String path) {
-    if (path.startsWith(watcher.assignmentZNode)) {
-      final String regionName = ZKAssign.getRegionName(watcher, path);
-      zkEventWorkersSubmit(new RegionRunnable() {
-        @Override
-        public String getRegionName() {
-          return regionName;
-        }
-
-        @Override
-        public void run() {
-          Lock lock = locker.acquireLock(regionName);
-          try {
-            RegionState rs = regionStates.getRegionTransitionState(regionName);
-            if (rs == null) {
-              rs = regionStates.getRegionState(regionName);
-              if (rs == null || !rs.isMergingNew()) {
-                // MergingNew is an offline state
-                return;
-              }
-            }
-
-            HRegionInfo regionInfo = rs.getRegion();
-            String regionNameStr = regionInfo.getRegionNameAsString();
-            LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
-
-            boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
-                ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
-
-            ServerName serverName = rs.getServerName();
-            if (serverManager.isServerOnline(serverName)) {
-              if (rs.isOnServer(serverName)
-                  && (rs.isOpened() || rs.isSplitting())) {
-                regionOnline(regionInfo, serverName);
-                if (disabled) {
-                  // if server is offline, no hurt to unassign again
-                  LOG.info("Opened " + regionNameStr
-                    + "but this table is disabled, triggering close of region");
-                  unassign(regionInfo);
-                }
-              } else if (rs.isMergingNew()) {
-                synchronized (regionStates) {
-                  String p = regionInfo.getEncodedName();
-                  PairOfSameType<HRegionInfo> regions = mergingRegions.get(p);
-                  if (regions != null) {
-                    onlineMergingRegion(disabled, regions.getFirst(), serverName);
-                    onlineMergingRegion(disabled, regions.getSecond(), serverName);
-                  }
-                }
-              }
-            }
-          } finally {
-            lock.unlock();
-          }
-        }
-
-        private void onlineMergingRegion(boolean disabled,
-            final HRegionInfo hri, final ServerName serverName) {
-          RegionState regionState = regionStates.getRegionState(hri);
-          if (regionState != null && regionState.isMerging()
-              && regionState.isOnServer(serverName)) {
-            regionOnline(regionState.getRegion(), serverName);
-            if (disabled) {
-              unassign(hri);
-            }
-          }
-        }
-      });
-    }
-  }
-
-  /**
-   * New unassigned node has been created.
-   *
-   * <p>This happens when an RS begins the OPENING, SPLITTING or CLOSING of a
-   * region by creating a znode.
-   *
-   * <p>When this happens we must:
-   * <ol>
-   *   <li>Watch the node for further children changed events</li>
-   *   <li>Watch all new children for changed events</li>
-   * </ol>
-   */
-  @Override
-  public void nodeChildrenChanged(String path) {
-    if (path.equals(watcher.assignmentZNode)) {
-      zkEventWorkers.submit(new Runnable() {
-        @Override
-        public void run() {
-          try {
-            // Just make sure we see the changes for the new znodes
-            List<String> children =
-              ZKUtil.listChildrenAndWatchForNewChildren(
-                watcher, watcher.assignmentZNode);
-            if (children != null) {
-              Stat stat = new Stat();
-              for (String child : children) {
-                // if region is in transition, we already have a watch
-                // on it, so no need to watch it again. So, as I know for now,
-                // this is needed to watch splitting nodes only.
-                if (!regionStates.isRegionInTransition(child)) {
-                  ZKAssign.getDataAndWatch(watcher, child, stat);
-                }
-              }
-            }
-          } catch (KeeperException e) {
-            server.abort("Unexpected ZK exception reading unassigned children", e);
-          }
-        }
-      });
-    }
-  }
-
-
-  /**
    * Marks the region as online.  Removes it from regions in transition and
    * updates the in-memory assignment information.
    * <p>
@@ -1432,55 +601,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * Pass the assignment event to a worker for processing.
-   * Each worker is a single thread executor service.  The reason
-   * for just one thread is to make sure all events for a given
-   * region are processed in order.
-   *
-   * @param path
-   */
-  private void handleAssignmentEvent(final String path) {
-    if (path.startsWith(watcher.assignmentZNode)) {
-      final String regionName = ZKAssign.getRegionName(watcher, path);
-
-      zkEventWorkersSubmit(new RegionRunnable() {
-        @Override
-        public String getRegionName() {
-          return regionName;
-        }
-
-        @Override
-        public void run() {
-          try {
-            Stat stat = new Stat();
-            byte [] data = ZKAssign.getDataAndWatch(watcher, path, stat);
-            if (data == null) return;
-
-            RegionTransition rt = RegionTransition.parseFrom(data);
-
-            // TODO: This code is tied to ZK anyway, so for now leaving it as is,
-            // will refactor when whole region assignment will be abstracted from ZK
-            BaseCoordinatedStateManager csm =
-              (BaseCoordinatedStateManager) server.getCoordinatedStateManager();
-            OpenRegionCoordination openRegionCoordination = csm.getOpenRegionCoordination();
-
-            ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
-              new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-            zkOrd.setVersion(stat.getVersion());
-            zkOrd.setServerName(csm.getServer().getServerName());
-
-            handleRegion(rt, openRegionCoordination, zkOrd);
-          } catch (KeeperException e) {
-            server.abort("Unexpected ZK exception reading unassigned node data", e);
-          } catch (DeserializationException e) {
-            server.abort("Unexpected exception deserializing node data", e);
-          }
-        }
-      });
-    }
-  }
-
-  /**
    * Marks the region as offline.  Removes it from regions in transition and
    * removes in-memory assignment information.
    * <p>
@@ -1492,15 +612,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   public void offlineDisabledRegion(HRegionInfo regionInfo) {
-    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);
-    }
     replicasToClose.remove(regionInfo);
     regionOffline(regionInfo);
   }
@@ -1517,23 +628,19 @@ public class AssignmentManager extends ZooKeeperListener {
    * Updates the RegionState and sends the OPEN RPC.
    * <p>
    * This will only succeed if the region is in transition and in a CLOSED or
-   * OFFLINE state or not in transition (in-memory not zk), and of course, the
-   * chosen server is up and running (It may have just crashed!).  If the
-   * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
+   * OFFLINE state or not in transition, and of course, the
+   * chosen server is up and running (It may have just crashed!).
    *
    * @param region server to be assigned
-   * @param setOfflineInZK whether ZK node should be created/transitioned to an
-   *                       OFFLINE state before assigning the region
    */
-  public void assign(HRegionInfo region, boolean setOfflineInZK) {
-    assign(region, setOfflineInZK, false);
+  public void assign(HRegionInfo region) {
+    assign(region, false);
   }
 
   /**
    * Use care with forceNewPlan. It could cause double assignment.
    */
-  public void assign(HRegionInfo region,
-      boolean setOfflineInZK, boolean forceNewPlan) {
+  public void assign(HRegionInfo region, boolean forceNewPlan) {
     if (isDisabledorDisablingRegionInRIT(region)) {
       return;
     }
@@ -1553,7 +660,7 @@ public class AssignmentManager extends ZooKeeperListener {
             + " is dead but not processed yet");
           return;
         }
-        assign(state, setOfflineInZK && useZKForAssignment, forceNewPlan);
+        assign(state, forceNewPlan);
       }
     } finally {
       lock.unlock();
@@ -1583,12 +690,8 @@ public class AssignmentManager extends ZooKeeperListener {
       List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
       Map<String, Lock> locks = locker.acquireLocks(encodedNames);
       try {
-        AtomicInteger counter = new AtomicInteger(0);
-        Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
-        OfflineCallback cb = new OfflineCallback(
-          watcher, destination, counter, offlineNodesVersions);
-        Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
-        List<RegionState> states = new ArrayList<RegionState>(regions.size());
+        Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regionCount);
+        List<RegionState> states = new ArrayList<RegionState>(regionCount);
         for (HRegionInfo region : regions) {
           String encodedName = region.getEncodedName();
           if (!isDisabledorDisablingRegionInRIT(region)) {
@@ -1600,8 +703,7 @@ public class AssignmentManager extends ZooKeeperListener {
                   + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
                   + " is dead but not processed yet");
                 onDeadServer = true;
-              } else if (!useZKForAssignment
-                  || asyncSetOfflineInZooKeeper(state, cb, destination)) {
+              } else {
                 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
                 plans.put(encodedName, plan);
                 states.add(state);
@@ -1610,8 +712,8 @@ public class AssignmentManager extends ZooKeeperListener {
             }
             // Reassign if the region wasn't on a dead server
             if (!onDeadServer) {
-              LOG.info("failed to force region state to offline or "
-                + "failed to set it offline in ZK, will reassign later: " + region);
+              LOG.info("failed to force region state to offline, "
+                + "will reassign later: " + region);
               failedToOpenRegions.add(region); // assign individually later
             }
           }
@@ -1621,21 +723,6 @@ public class AssignmentManager extends ZooKeeperListener {
           lock.unlock();
         }
 
-        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.debug(destination.toString() + " unassigned znodes=" + count +
-                " of total=" + total + "; oldCounter=" + oldCounter);
-              oldCounter = count;
-            }
-            if (count >= total) break;
-            Thread.sleep(5);
-          }
-        }
-
         if (server.isStopped()) {
           return false;
         }
@@ -1644,27 +731,18 @@ public class AssignmentManager extends ZooKeeperListener {
         // that unnecessary timeout on RIT is reduced.
         this.addPlans(plans);
 
-        List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
-          new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
+        List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos =
+          new ArrayList<Pair<HRegionInfo, List<ServerName>>>(states.size());
         for (RegionState state: states) {
           HRegionInfo region = state.getRegion();
-          String encodedRegionName = region.getEncodedName();
-          Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
-          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);
-            lock.unlock();
-          } else {
-            regionStates.updateRegionState(
-              region, State.PENDING_OPEN, destination);
-            List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
-            if (this.shouldAssignRegionsWithFavoredNodes) {
-              favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
-            }
-            regionOpenInfos.add(new Triple<HRegionInfo, Integer,  List<ServerName>>(
-              region, nodeVersion, favoredNodes));
+          regionStates.updateRegionState(
+            region, State.PENDING_OPEN, destination);
+          List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
+          if (this.shouldAssignRegionsWithFavoredNodes) {
+            favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
           }
+          regionOpenInfos.add(new Pair<HRegionInfo, List<ServerName>>(
+            region, favoredNodes));
         }
 
         // Move on to open regions.
@@ -1686,15 +764,8 @@ public class AssignmentManager extends ZooKeeperListener {
                 RegionOpeningState openingState = regionOpeningStateList.get(k);
                 if (openingState != RegionOpeningState.OPENED) {
                   HRegionInfo region = regionOpenInfos.get(k).getFirst();
-                  if (openingState == RegionOpeningState.ALREADY_OPENED) {
-                    processAlreadyOpenedRegion(region, destination);
-                  } else if (openingState == RegionOpeningState.FAILED_OPENING) {
-                    // Failed opening this region, reassign it later
-                    failedToOpenRegions.add(region);
-                  } else {
-                    LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
-                      + openingState + " in assigning region " + region);
-                  }
+                  // Failed opening this region, reassign it later
+                  failedToOpenRegions.add(region);
                 }
               }
               break;
@@ -1771,8 +842,7 @@ public class AssignmentManager extends ZooKeeperListener {
    * on an unexpected server scenario, for an example)
    */
   private void unassign(final HRegionInfo region,
-      final RegionState state, final int versionOfClosingNode,
-      final ServerName dest, final boolean transitionInZK,
+      final RegionState state, final ServerName dest,
       final ServerName src) {
     ServerName server = src;
     if (state != null) {
@@ -1788,10 +858,6 @@ public class AssignmentManager extends ZooKeeperListener {
       if (!serverManager.isServerOnline(server)) {
         LOG.debug("Offline " + region.getRegionNameAsString()
           + ", no need to unassign since it's on a dead server: " + server);
-        if (transitionInZK) {
-          // delete the node. if no node exists need not bother.
-          deleteClosingOrClosedNode(region, server);
-        }
         if (state != null) {
           regionOffline(region);
         }
@@ -1799,16 +865,9 @@ public class AssignmentManager extends ZooKeeperListener {
       }
       try {
         // Send CLOSE RPC
-        if (serverManager.sendRegionClose(server, region,
-          versionOfClosingNode, dest, transitionInZK)) {
+        if (serverManager.sendRegionClose(server, region, dest)) {
           LOG.debug("Sent CLOSE to " + server + " for region " +
             region.getRegionNameAsString());
-          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);
-          }
           return;
         }
         // This never happens. Currently regionserver close always return true.
@@ -1825,9 +884,6 @@ public class AssignmentManager extends ZooKeeperListener {
             || t instanceof ServerNotRunningYetException) {
           LOG.debug("Offline " + region.getRegionNameAsString()
             + ", it's not any more on " + server, t);
-          if (transitionInZK) {
-            deleteClosingOrClosedNode(region, server);
-          }
           if (state != null) {
             regionOffline(region);
           }
@@ -1840,9 +896,6 @@ public class AssignmentManager extends ZooKeeperListener {
             sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
                   RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
           } else {
-            // RS is already processing this region, only need to update the timestamp
-            LOG.debug("update " + state + " the timestamp.");
-            state.updateTimestampToNow();
             if (maxWaitTime < 0) {
               maxWaitTime =
                   EnvironmentEdgeManager.currentTimeMillis()
@@ -1898,7 +951,6 @@ public class AssignmentManager extends ZooKeeperListener {
       state = regionStates.createRegionState(region);
     }
 
-    ServerName sn = state.getServerName();
     if (forceNewPlan && LOG.isDebugEnabled()) {
       LOG.debug("Force region state offline " + state);
     }
@@ -1916,7 +968,7 @@ public class AssignmentManager extends ZooKeeperListener {
       }
     case FAILED_CLOSE:
     case FAILED_OPEN:
-      unassign(region, state, -1, null, false, null);
+      unassign(region, state, null, null);
       state = regionStates.getRegionState(region);
       if (state.isFailedClose()) {
         // If we can't close the region, we can't re-assign
@@ -1926,21 +978,6 @@ public class AssignmentManager extends ZooKeeperListener {
         return null;
       }
     case OFFLINE:
-      // This region could have been open on this server
-      // for a while. If the server is dead and not processed
-      // 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.
-      // 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: " + sn);
-        return null;
-      }
     case CLOSED:
       break;
     default:
@@ -1951,49 +988,15 @@ public class AssignmentManager extends ZooKeeperListener {
     return state;
   }
 
-  @SuppressWarnings("deprecation")
-  private boolean wasRegionOnDeadServerByMeta(
-      final HRegionInfo region, final ServerName sn) {
-    try {
-      if (region.isMetaRegion()) {
-        ServerName server = this.server.getMetaTableLocator().
-          getMetaRegionLocation(this.server.getZooKeeper());
-        return regionStates.isServerDeadAndNotProcessed(server);
-      }
-      while (!server.isStopped()) {
-        try {
-          this.server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
-          Result r = MetaTableAccessor.getRegionResult(server.getShortCircuitConnection(),
-            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 "
-            + region.getRegionNameAsString() + ", retrying", ioe);
-        }
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.info("Interrupted accessing hbase:meta", e);
-    }
-    // Call is interrupted or server is stopped.
-    return regionStates.isServerDeadAndNotProcessed(sn);
-  }
-
   /**
    * Caller must hold lock on the passed <code>state</code> object.
    * @param state
-   * @param setOfflineInZK
    * @param forceNewPlan
    */
-  private void assign(RegionState state,
-      final boolean setOfflineInZK, final boolean forceNewPlan) {
+  private void assign(RegionState state, boolean forceNewPlan) {
     long startTime = EnvironmentEdgeManager.currentTimeMillis();
     try {
       Configuration conf = server.getConfiguration();
-      RegionState currentState = state;
-      int versionOfOfflineNode = -1;
       RegionPlan plan = null;
       long maxWaitTime = -1;
       HRegionInfo region = state.getRegion();
@@ -2027,14 +1030,6 @@ public class AssignmentManager extends ZooKeeperListener {
           regionStates.updateRegionState(region, State.FAILED_OPEN);
           return;
         }
-        if (setOfflineInZK && versionOfOfflineNode == -1) {
-          // get the version of the znode after setting it to OFFLINE.
-          // versionOfOfflineNode will be -1 if the znode was not set to OFFLINE
-          versionOfOfflineNode = setOfflineInZooKeeper(currentState, plan.getDestination());
-          if (versionOfOfflineNode != -1) {
-            if (isDisabledorDisablingRegionInRIT(region)) {
-              return;
-            }
             // In case of assignment from EnableTableHandler table state is ENABLING. Any how
             // EnableTableHandler will set ENABLED after assigning all the table regions. If we
             // try to set to ENABLED directly then client API may think table is enabled.
@@ -2047,22 +1042,10 @@ public class AssignmentManager extends ZooKeeperListener {
               LOG.debug("Setting table " + tableName + " to ENABLED state.");
               setEnabledTable(tableName);
             }
-          }
-        }
-        if (setOfflineInZK && versionOfOfflineNode == -1) {
-          LOG.info("Unable to set offline in ZooKeeper to assign " + region);
-          // Setting offline in ZK must have been failed due to ZK racing or some
-          // exception which may make the server to abort. If it is ZK racing,
-          // we should retry since we already reset the region state,
-          // existing (re)assignment will fail anyway.
-          if (!server.isAborted()) {
-            continue;
-          }
-        }
         LOG.info("Assigning " + region.getRegionNameAsString() +
             " to " + plan.getDestination().toString());
         // Transition RegionState to PENDING_OPEN
-        currentState = regionStates.updateRegionState(region,
+       regionStates.updateRegionState(region,
           State.PENDING_OPEN, plan.getDestination());
 
         boolean needNewPlan;
@@ -2074,7 +1057,7 @@ public class AssignmentManager extends ZooKeeperListener {
             favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
           }
           regionOpenState = serverManager.sendRegionOpen(
-              plan.getDestination(), region, versionOfOfflineNode, favoredNodes);
+              plan.getDestination(), region, favoredNodes);
 
           if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
             // Failed opening this region, looping again on a new server.
@@ -2084,9 +1067,6 @@ public class AssignmentManager extends ZooKeeperListener {
                 "try=" + i + " of " + this.maximumAttempts);
           } else {
             // we're done
-            if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
-              processAlreadyOpenedRegion(region, plan.getDestination());
-            }
             return;
           }
 
@@ -2186,8 +1166,7 @@ public class AssignmentManager extends ZooKeeperListener {
             // Clean out plan we failed execute and one that doesn't look like it'll
             // succeed anyways; we need a new plan!
             // Transition back to OFFLINE
-            currentState = regionStates.updateRegionState(region, State.OFFLINE);
-            versionOfOfflineNode = -1;
+            regionStates.updateRegionState(region, State.OFFLINE);
             plan = newPlan;
           } else if(plan.getDestination().equals(newPlan.getDestination()) &&
               previousException instanceof FailedServerException) {
@@ -2213,17 +1192,6 @@ public class AssignmentManager extends ZooKeeperListener {
     }
   }
 
-  private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
-    // Remove region from in-memory transition and unassigned node from ZK
-    // While trying to enable the table the regions of the table were
-    // already enabled.
-    LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
-      + " to " + sn);
-    String encodedName = region.getEncodedName();
-    deleteNodeInStates(encodedName, "offline", sn, EventType.M_ZK_REGION_OFFLINE);
-    regionStates.regionOnline(region, sn);
-  }
-
   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
     if (this.tableStateManager.isTableState(region.getTable(),
         ZooKeeperProtos.Table.State.DISABLED,
@@ -2237,37 +1205,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * Set region as OFFLINED up in zookeeper
-   *
-   * @param state
-   * @return the version of the offline node if setting of the OFFLINE node was
-   *         successful, -1 otherwise.
-   */
-  private int setOfflineInZooKeeper(final RegionState state, final ServerName destination) {
-    if (!state.isClosed() && !state.isOffline()) {
-      String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
-      this.server.abort(msg, new IllegalStateException(msg));
-      return -1;
-    }
-    regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
-    int versionOfOfflineNode;
-    try {
-      // get the version after setting the znode to OFFLINE
-      versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(watcher,
-        state.getRegion(), destination);
-      if (versionOfOfflineNode == -1) {
-        LOG.warn("Attempted to create/force node into OFFLINE state before "
-            + "completing assignment but failed to do so for " + state);
-        return -1;
-      }
-    } catch (KeeperException e) {
-      server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
-      return -1;
-    }
-    return versionOfOfflineNode;
-  }
-
-  /**
    * @param region the region to assign
    * @return Plan for passed <code>region</code> (If none currently, it creates one or
    * if no servers to assign, it returns null).
@@ -2388,7 +1325,6 @@ public class AssignmentManager extends ZooKeeperListener {
 
     String encodedName = region.getEncodedName();
     // Grab the state of this region and synchronize on it
-    int versionOfClosingNode = -1;
     // We need a lock here as we're going to do a put later and we don't want multiple states
     //  creation
     ReentrantLock lock = locker.acquireLock(encodedName);
@@ -2404,56 +1340,12 @@ public class AssignmentManager extends ZooKeeperListener {
           // Offline region will be reassigned below
           return;
         }
-        // Create the znode in CLOSING state
-        try {
-          if (state == null || state.getServerName() == null) {
-            // We don't know where the region is, offline it.
-            // No need to send CLOSE RPC
-            LOG.warn("Attempting to unassign a region not in RegionStates"
-              + region.getRegionNameAsString() + ", offlined");
-            regionOffline(region);
-            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) {
-            // Handle race between master initiated close and regionserver
-            // orchestrated splitting. See if existing node is in a
-            // SPLITTING or SPLIT state.  If so, the regionserver started
-            // an op on node before we could get our CLOSING in.  Deal.
-            NodeExistsException nee = (NodeExistsException)e;
-            String path = nee.getPath();
-            try {
-              if (isSplitOrSplittingOrMergedOrMerging(path)) {
-                LOG.debug(path + " is SPLIT or SPLITTING or MERGED or MERGING; " +
-                  "skipping unassign because region no longer exists -- its split or merge");
-                reassign = false; // no need to reassign for split/merged region
-                return;
-              }
-            } catch (KeeperException.NoNodeException ke) {
-              LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
-                "; presuming split and that the region to unassign, " +
-                encodedName + ", no longer exists -- confirm", ke);
-              return;
-            } catch (KeeperException ke) {
-              LOG.error("Unexpected zk state", ke);
-            } catch (DeserializationException de) {
-              LOG.error("Failed parse", de);
-            }
-          }
-          // If we get here, don't understand whats going on -- abort.
-          server.abort("Unexpected ZK exception creating node CLOSING", e);
-          reassign = false; // heading out already
+        if (state == null || state.getServerName() == null) {
+          // We don't know where the region is, offline it.
+          // No need to send CLOSE RPC
+          LOG.warn("Attempting to unassign a region not in RegionStates"
+            + region.getRegionNameAsString() + ", offlined");
+          regionOffline(region);
           return;
         }
         state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
@@ -2468,7 +1360,6 @@ public class AssignmentManager extends ZooKeeperListener {
         if (state.isFailedClose()) {
           state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
         }
-        state.updateTimestampToNow();
       } else {
         LOG.debug("Attempting to unassign " +
           region.getRegionNameAsString() + " but it is " +
@@ -2476,13 +1367,13 @@ public class AssignmentManager extends ZooKeeperListener {
         return;
       }
 
-      unassign(region, state, versionOfClosingNode, dest, useZKForAssignment, null);
+      unassign(region, state, dest, null);
     } finally {
       lock.unlock();
 
       // Region is expected to be reassigned afterwards
       if (!replicasToClose.contains(region) && reassign && regionStates.isRegionOffline(region)) {
-        assign(region, true);
+        assign(region);
       }
     }
   }
@@ -2492,48 +1383,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * @param region regioninfo of znode to be deleted.
-   */
-  public void deleteClosingOrClosedNode(HRegionInfo region, ServerName sn) {
-    String encodedName = region.getEncodedName();
-    deleteNodeInStates(encodedName, "closing", sn, EventType.M_ZK_REGION_CLOSING,
-      EventType.RS_ZK_REGION_CLOSED);
-  }
-
-  /**
-   * @param path
-   * @return True if znode is in SPLIT or SPLITTING or MERGED or MERGING state.
-   * @throws KeeperException Can happen if the znode went away in meantime.
-   * @throws DeserializationException
-   */
-  private boolean isSplitOrSplittingOrMergedOrMerging(final String path)
-      throws KeeperException, DeserializationException {
-    boolean result = false;
-    // This may fail if the SPLIT or SPLITTING or MERGED or MERGING znode gets
-    // cleaned up before we can get data from it.
-    byte [] data = ZKAssign.getData(watcher, path);
-    if (data == null) {
-      LOG.info("Node " + path + " is gone");
-      return false;
-    }
-    RegionTransition rt = RegionTransition.parseFrom(data);
-    switch (rt.getEventType()) {
-    case RS_ZK_REQUEST_REGION_SPLIT:
-    case RS_ZK_REGION_SPLIT:
-    case RS_ZK_REGION_SPLITTING:
-    case RS_ZK_REQUEST_REGION_MERGE:
-    case RS_ZK_REGION_MERGED:
-    case RS_ZK_REGION_MERGING:
-      result = true;
-      break;
-    default:
-      LOG.info("Node " + path + " is in " + rt.getEventType());
-      break;
-    }
-    return result;
-  }
-
-  /**
    * Used by unit tests. Return the number of regions opened so far in the life
    * of the master. Increases by one every time the master opens a region
    * @return the counter value of the number of regions opened so far
@@ -2577,8 +1426,8 @@ public class AssignmentManager extends ZooKeeperListener {
    * @throws KeeperException
    */
   public void assignMeta() throws KeeperException {
-    this.server.getMetaTableLocator().deleteMetaLocation(this.watcher);
-    assign(HRegionInfo.FIRST_META_REGIONINFO, true);
+    this.server.getMetaTableLocator().deleteMetaLocation(this.server.getZooKeeper());
+    assign(HRegionInfo.FIRST_META_REGIONINFO);
   }
 
   /**
@@ -2735,30 +1584,6 @@ public class AssignmentManager extends ZooKeeperListener {
   }
 
   /**
-   * Wait until no regions in transition.
-   * @param timeout How long to wait.
-   * @return True if nothing in regions in transition.
-   * @throws InterruptedException
-   */
-  boolean waitUntilNoRegionsInTransition(final long timeout)
-      throws InterruptedException {
-    // Blocks until there are no regions in transition. It is possible that
-    // there
-    // are regions in transition immediately after this returns but guarantees
-    // that if it returns without an exception that there was a period of time
-    // with no regions in transition from the point-of-view of the in-memory
-    // state of the Master.
-    final long endTime = System.currentTimeMillis() + timeout;
-
-    while (!this.server.isStopped() && regionStates.isRegionsInTransition()
-        && endTime > System.currentTimeMillis()) {
-      regionStates.waitForUpdate(100);
-    }
-
-    return !regionStates.isRegionsInTransition();
-  }
-
-  /**
    * Rebuild the list of user regions and assignment information.
    * <p>
    * Returns a set of servers that are not found to be online that hosted
@@ -2829,16 +1654,11 @@ public class AssignmentManager extends ZooKeeperListener {
         if (!onlineServers.contains(regionLocation)) {
           // Region is located on a server that isn't online
           offlineServers.add(regionLocation);
-          if (useZKForAssignment) {
-            regionStates.regionOffline(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
           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
@@ -2911,21 +1731,12 @@ public class AssignmentManager extends ZooKeeperListener {
 
   /**
    * Processes list of dead servers from result of hbase:meta scan and regions in RIT
-   * <p>
-   * This is used for failover to recover the lost regions that belonged to
-   * RegionServers which failed while there was no active master or regions
-   * that were in RIT.
-   * <p>
-   *
    *
    * @param deadServers
    *          The list of dead servers which failed while there was no active
    *          master. Can be null.
-   * @throws IOException
-   * @throws KeeperException
    */
-  private void processDeadServersAndRecoverLostRegions(
-      Set<ServerName> deadServers) throws IOException, KeeperException {
+  private void processDeadServers(Set<ServerName> deadServers) {
     if (deadServers != null && !deadServers.isEmpty()) {
       for (ServerName serverName: deadServers) {
         if (!serverManager.isServerDead(serverName)) {
@@ -2934,36 +1745,27 @@ public class AssignmentManager extends ZooKeeperListener {
       }
     }
 
-    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);
+    // 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
       }
-    } 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
-        }
+      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
       }
     }
   }
@@ -2992,7 +1794,7 @@ public class AssignmentManager extends ZooKeeperListener {
                   favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri);
                 }
                 RegionOpeningState regionOpenState = serverManager.sendRegionOpen(
-                  serverName, hri, -1, favoredNodes);
+                  serverName, hri, favoredNodes);
 
                 if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
                   // Failed opening this region, this means the target server didn't get
@@ -3045,7 +1847,7 @@ public class AssignmentManager extends ZooKeeperListener {
             while (serverManager.isServerOnline(serverName)
                 && !server.isStopped() && !server.isAborted()) {
               try {
-                if (!serverManager.sendRegionClose(serverName, hri, -1, null, false)) {
+                if (!serverManager.sendRegionClose(serverName, hri, null)) {
                   // This means the region is still on the target server
                   LOG.debug("Got false in retry sendRegionClose for "
                     + regionState + ", re-close it");
@@ -3179,43 +1981,26 @@ public class AssignmentManager extends ZooKeeperListener {
 
   /**
    * Check if the shutdown server carries the specific region.
-   * We have a bunch of places that store region location
-   * Those values aren't consistent. There is a delay of notification.
-   * The location from zookeeper unassigned node has the most recent data;
-   * but the node could be deleted after the region is opened by AM.
-   * The AM's info could be old when OpenedRegionHandler
-   * processing hasn't finished yet when server shutdown occurs.
    * @return whether the serverName currently hosts the region
    */
   private boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
-    RegionTransition rt = null;
-    try {
-      byte [] data = ZKAssign.getData(watcher, hri.getEncodedName());
-      // This call can legitimately come by null
-      rt = data == null? null: RegionTransition.parseFrom(data);
-    } catch (KeeperException e) {
-      server.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
-    } catch (DeserializationException e) {
-      server.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
-    }
-
-    ServerName addressFromZK = rt != null? rt.getServerName():  null;
-    if (addressFromZK != null) {
-      // if we get something from ZK, we will use the data
-      boolean matchZK = addressFromZK.equals(serverName);
-      LOG.debug("Checking region=" + hri.getRegionNameAsString() + ", zk server=" + addressFromZK +
-        " current=" + serverName + ", matches=" + matchZK);
-      return matchZK;
+    RegionState regionState = regionStates.getRegionTransitionState(hri);
+    ServerName transitionAddr = regionState != null? regionState.getServerName(): null;
+    if (transitionAddr != null) {
+      boolean matchTransitionAddr = transitionAddr.equals(serverName);
+      LOG.debug("Checking region=" + hri.getRegionNameAsString()
+        + ", transitioning on server=" + matchTransitionAddr
+        + " server being checked: " + serverName
+        + ", matches=" + matchTransitionAddr);
+      return matchTransitionAddr;
     }
 
-    ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
-    boolean matchAM = (addressFromAM != null &&
-      addressFromAM.equals(serverName));
-    LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
-      " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
-      " server being checked: " + serverName);
-
-    return matchAM;
+    ServerName assignedAddr = regionStates.getRegionServerOfRegion(hri);
+    boolean matchAssignedAddr = serverName.equals(assignedAddr);
+    LOG.debug("based on AM, current region=" + hri.getRegionNameAsString()
+      + " is on server=" + assignedAddr + ", server being checked: "
+      + serverName);
+    return matchAssignedAddr;
   }
 
   /**
@@ -3237,7 +2022,7 @@ public class AssignmentManager extends ZooKeeperListener {
         }
       }
     }
-    List<HRegionInfo> regions = regionStates.serverOffline(watcher, sn);
+    List<HRegionInfo> regions = regionStates.serverOffline(sn);
     for (Iterator<HRegionInfo> it = regions.iterator(); it.hasNext(); ) {
       HRegionInfo hri = it.next();
       String encodedName = hri.getEncodedName();
@@ -3255,12 +2040,6 @@ public class AssignmentManager extends ZooKeeperListener {
             + " on the dead server any more: " + sn);
           it.remove();
         } else {
-          try {
-            // Delete the ZNode if exists
-            ZKAssign.deleteNodeFailSilent(watcher, hri);
-          } catch (KeeperException ke) {
-            server.abort("Unexpected ZK exception deleting node " + hri, ke);
-          }
           if (tableStateManager.isTableState(hri.getTable(),
               ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
             regionStates.regionOffline(hri);
@@ -3317,12 +2096,7 @@ public class AssignmentManager extends ZooKeeperListener {
    * Shutdown the threadpool executor service
    */
   public void shutdown() {
-    // It's an immediate shutdown, so we're clearing the remaining tasks.
-    synchronized (zkEventWorkerWaitingList){
-      zkEventWorkerWaitingList.clear();
-    }
     threadPoolExecutorService.shutdownNow();
-    zkEventWorkers.shutdownNow();
     regionStateStore.stop();
   }
 
@@ -3339,65 +2113,6 @@ public class AssignmentManager extends ZooKeeperListener {
     }
   }
 
-  /**
-   * Set region as OFFLINED up in zookeeper asynchronously.
-   * @param state
-   * @return True if we succeeded, false otherwise (State was incorrect or failed
-   * updating zk).
-   */
-  private boolean asyncSetOfflineInZooKeeper(final RegionState state,
-      final AsyncCallback.StringCallback cb, final ServerName destination) {
-    if (!state.isClosed() && !state.isOffline()) {
-      this.server.abort("Unexpected state trying to OFFLINE; " + state,
-        new IllegalStateException());
-      return false;
-    }
-    regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
-    try {
-      ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
-        destination, cb, state);
-    } catch (KeeperException e) {
-      if (e instanceof NodeExistsException) {
-        LOG.warn("Node for " + state.getRegion() + " already exists");
-      } else {
-        server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
-      }
-      return false;
-    }
-    return true;
-  }
-
-  private boolean deleteNodeInStates(String encodedName,
-      String desc, ServerName sn, EventType... types) {
-    try {
-      for (EventType et: types) {
-        if (ZKAssign.deleteNode(watcher, encodedName, et, sn)) {
-          return true;
-        }
-      }
-      LOG.info("Failed to delete the " + desc + " node for "
-        + encodedName + ". The node type may not match");
-    } catch (NoNodeException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("The " + desc + " node for " + encodedName + " already deleted");
-      }
-    } catch (KeeperException ke) {
-      server.abort("Unexpected ZK exception deleting " + desc
-        + " node for the region " + encodedName, ke);
-    }
-    return false;
-  }
-
-  private void deleteMergingNode(String encodedName, ServerName sn) {
-    deleteNodeInStates

<TRUNCATED>

[04/10] HBASE-11611 Clean up ZK-based region assignment

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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
deleted file mode 100644
index 4a3d4f2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
+++ /dev/null
@@ -1,1475 +0,0 @@
-/**
- * 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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.hbase.CellScannable;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.RegionException;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.MetaMockingUtil;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
-import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.executor.ExecutorType;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
-import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
-import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer;
-import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
-import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
-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.ZooKeeperProtos.Table;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.Watcher;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.internal.util.reflection.Whitebox;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
-
-/**
- * Test {@link AssignmentManager}
- */
-@Category(MediumTests.class)
-public class TestAssignmentManager {
-  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
-  private static final ServerName SERVERNAME_A =
-      ServerName.valueOf("example.org", 1234, 5678);
-  private static final ServerName SERVERNAME_B =
-      ServerName.valueOf("example.org", 0, 5678);
-  private static final HRegionInfo REGIONINFO =
-    new HRegionInfo(TableName.valueOf("t"),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
-  private static int assignmentCount;
-  private static boolean enabling = false;
-
-  // Mocked objects or; get redone for each test.
-  private Server server;
-  private ServerManager serverManager;
-  private ZooKeeperWatcher watcher;
-  private CoordinatedStateManager cp;
-  private MetaTableLocator mtl;
-  private LoadBalancer balancer;
-  private HMaster master;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
-    HTU.startMiniZKCluster();
-  }
-
-  @AfterClass
-  public static void afterClass() throws IOException {
-    HTU.shutdownMiniZKCluster();
-  }
-
-  @Before
-  public void before() throws ZooKeeperConnectionException, IOException {
-    // TODO: Make generic versions of what we do below and put up in a mocking
-    // utility class or move up into HBaseTestingUtility.
-
-    // Mock a Server.  Have it return a legit Configuration and ZooKeeperWatcher.
-    // If abort is called, be sure to fail the test (don't just swallow it
-    // silently as is mockito default).
-    this.server = Mockito.mock(Server.class);
-    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
-    Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
-    this.watcher =
-      new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
-    Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);
-    Mockito.doThrow(new RuntimeException("Aborted")).
-      when(server).abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
-
-    cp = new ZkCoordinatedStateManager();
-    cp.initialize(this.server);
-    cp.start();
-
-    mtl = Mockito.mock(MetaTableLocator.class);
-
-    Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
-    Mockito.when(server.getMetaTableLocator()).thenReturn(mtl);
-
-    // Get a connection w/ mocked up common methods.
-    HConnection connection =
-      HConnectionTestingUtility.getMockedConnection(HTU.getConfiguration());
-
-    // Make it so we can get a catalogtracker from servermanager.. .needed
-    // down in guts of server shutdown handler.
-    Mockito.when(server.getShortCircuitConnection()).thenReturn(connection);
-
-    // Mock a ServerManager.  Say server SERVERNAME_{A,B} are online.  Also
-    // make it so if close or open, we return 'success'.
-    this.serverManager = Mockito.mock(ServerManager.class);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(true);
-    Mockito.when(this.serverManager.getDeadServers()).thenReturn(new DeadServer());
-    final Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
-    onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
-    Mockito.when(this.serverManager.getOnlineServersList()).thenReturn(
-        new ArrayList<ServerName>(onlineServers.keySet()));
-    Mockito.when(this.serverManager.getOnlineServers()).thenReturn(onlineServers);
-
-    List<ServerName> avServers = new ArrayList<ServerName>();
-    avServers.addAll(onlineServers.keySet());
-    Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(avServers);
-    Mockito.when(this.serverManager.createDestinationServersList(null)).thenReturn(avServers);
-
-    Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, REGIONINFO, -1)).
-      thenReturn(true);
-    Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_B, REGIONINFO, -1)).
-      thenReturn(true);
-    // Ditto on open.
-    Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_A, REGIONINFO, -1, null)).
-      thenReturn(RegionOpeningState.OPENED);
-    Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_B, REGIONINFO, -1, null)).
-      thenReturn(RegionOpeningState.OPENED);
-    this.master = Mockito.mock(HMaster.class);
-
-    Mockito.when(this.master.getServerManager()).thenReturn(serverManager);
-  }
-
-  @After
-    public void after() throws KeeperException {
-    if (this.watcher != null) {
-      // Clean up all znodes
-      ZKAssign.deleteAllNodes(this.watcher);
-      this.watcher.close();
-      this.cp.stop();
-    }
-  }
-
-  /**
-   * Test a balance going on at same time as a master failover
-   *
-   * @throws IOException
-   * @throws KeeperException
-   * @throws InterruptedException
-   * @throws DeserializationException
-   */
-  @Test(timeout = 60000)
-  public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
-      throws IOException, KeeperException, InterruptedException, ServiceException,
-      DeserializationException, CoordinatedStateException {
-    AssignmentManagerWithExtrasForTesting am =
-      setUpMockedAssignmentManager(this.server, this.serverManager);
-    try {
-      createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
-      startFakeFailedOverMasterAssignmentManager(am, this.watcher);
-      while (!am.processRITInvoked) Thread.sleep(1);
-      // As part of the failover cleanup, the balancing region plan is removed.
-      // So a random server will be used to open the region. For testing purpose,
-      // let's assume it is going to open on server b:
-      am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
-
-      Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
-
-      // Now fake the region closing successfully over on the regionserver; the
-      // regionserver will have set the region in CLOSED state. This will
-      // trigger callback into AM. The below zk close call is from the RS close
-      // region handler duplicated here because its down deep in a private
-      // method hard to expose.
-      int versionid =
-        ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
-      assertNotSame(versionid, -1);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
-      // Get current versionid else will fail on transition from OFFLINE to
-      // OPENING below
-      versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
-      assertNotSame(-1, versionid);
-      // This uglyness below is what the openregionhandler on RS side does.
-      versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
-        SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, versionid);
-      assertNotSame(-1, versionid);
-      // Move znode from OPENING to OPENED as RS does on successful open.
-      versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
-        SERVERNAME_B, versionid);
-      assertNotSame(-1, versionid);
-      am.gate.set(false);
-      // Block here until our znode is cleared or until this test times out.
-      ZKAssign.blockUntilNoRIT(watcher);
-    } finally {
-      am.getExecutorService().shutdown();
-      am.shutdown();
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testBalanceOnMasterFailoverScenarioWithClosedNode()
-      throws IOException, KeeperException, InterruptedException, ServiceException,
-        DeserializationException, CoordinatedStateException {
-    AssignmentManagerWithExtrasForTesting am =
-      setUpMockedAssignmentManager(this.server, this.serverManager);
-    try {
-      createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
-      startFakeFailedOverMasterAssignmentManager(am, this.watcher);
-      while (!am.processRITInvoked) Thread.sleep(1);
-      // As part of the failover cleanup, the balancing region plan is removed.
-      // So a random server will be used to open the region. For testing purpose,
-      // let's assume it is going to open on server b:
-      am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
-
-      Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
-
-      // Now fake the region closing successfully over on the regionserver; the
-      // regionserver will have set the region in CLOSED state. This will
-      // trigger callback into AM. The below zk close call is from the RS close
-      // region handler duplicated here because its down deep in a private
-      // method hard to expose.
-      int versionid =
-        ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
-      assertNotSame(versionid, -1);
-      am.gate.set(false);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
-      // Get current versionid else will fail on transition from OFFLINE to
-      // OPENING below
-      versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
-      assertNotSame(-1, versionid);
-      // This uglyness below is what the openregionhandler on RS side does.
-      versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
-          SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
-          EventType.RS_ZK_REGION_OPENING, versionid);
-      assertNotSame(-1, versionid);
-      // Move znode from OPENING to OPENED as RS does on successful open.
-      versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
-          SERVERNAME_B, versionid);
-      assertNotSame(-1, versionid);
-
-      // Block here until our znode is cleared or until this test timesout.
-      ZKAssign.blockUntilNoRIT(watcher);
-    } finally {
-      am.getExecutorService().shutdown();
-      am.shutdown();
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
-      throws IOException, KeeperException, InterruptedException, ServiceException,
-      DeserializationException, CoordinatedStateException {
-    AssignmentManagerWithExtrasForTesting am =
-      setUpMockedAssignmentManager(this.server, this.serverManager);
-    try {
-      createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
-      startFakeFailedOverMasterAssignmentManager(am, this.watcher);
-      while (!am.processRITInvoked) Thread.sleep(1);
-      // As part of the failover cleanup, the balancing region plan is removed.
-      // So a random server will be used to open the region. For testing purpose,
-      // let's assume it is going to open on server b:
-      am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
-
-      Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
-
-      // Now fake the region closing successfully over on the regionserver; the
-      // regionserver will have set the region in CLOSED state. This will
-      // trigger callback into AM. The below zk close call is from the RS close
-      // region handler duplicated here because its down deep in a private
-      // method hard to expose.
-      int versionid =
-        ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
-      assertNotSame(versionid, -1);
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
-      am.gate.set(false);
-      // Get current versionid else will fail on transition from OFFLINE to
-      // OPENING below
-      versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
-      assertNotSame(-1, versionid);
-      // This uglyness below is what the openregionhandler on RS side does.
-      versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
-          SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
-          EventType.RS_ZK_REGION_OPENING, versionid);
-      assertNotSame(-1, versionid);
-      // Move znode from OPENING to OPENED as RS does on successful open.
-      versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
-          SERVERNAME_B, versionid);
-      assertNotSame(-1, versionid);
-      // Block here until our znode is cleared or until this test timesout.
-      ZKAssign.blockUntilNoRIT(watcher);
-    } finally {
-      am.getExecutorService().shutdown();
-      am.shutdown();
-    }
-  }
-
-  private void createRegionPlanAndBalance(
-      final AssignmentManager am, final ServerName from,
-      final ServerName to, final HRegionInfo hri) throws RegionException {
-    // Call the balance function but fake the region being online first at
-    // servername from.
-    am.regionOnline(hri, from);
-    // Balance region from 'from' to 'to'. It calls unassign setting CLOSING state
-    // up in zk.  Create a plan and balance
-    am.balance(new RegionPlan(hri, from, to));
-  }
-
-  /**
-   * Tests AssignmentManager balance function.  Runs a balance moving a region
-   * from one server to another mocking regionserver responding over zk.
-   * @throws IOException
-   * @throws KeeperException
-   * @throws DeserializationException
-   */
-  @Test
-  public void testBalance() throws IOException, KeeperException, DeserializationException,
-      InterruptedException, CoordinatedStateException {
-    // Create and startup an executor.  This is used by AssignmentManager
-    // handling zk callbacks.
-    ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
-
-    // We need a mocked catalog tracker.
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
-        .getConfiguration());
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, executor, null, master.getTableLockManager());
-    am.failoverCleanupDone.set(true);
-    try {
-      // Make sure our new AM gets callbacks; once registered, can't unregister.
-      // Thats ok because we make a new zk watcher for each test.
-      this.watcher.registerListenerFirst(am);
-      // Call the balance function but fake the region being online first at
-      // SERVERNAME_A.  Create a balance plan.
-      am.regionOnline(REGIONINFO, SERVERNAME_A);
-      // Balance region from A to B.
-      RegionPlan plan = new RegionPlan(REGIONINFO, SERVERNAME_A, SERVERNAME_B);
-      am.balance(plan);
-
-      RegionStates regionStates = am.getRegionStates();
-      // Must be failed to close since the server is fake
-      assertTrue(regionStates.isRegionInTransition(REGIONINFO)
-        && regionStates.isRegionInState(REGIONINFO, State.FAILED_CLOSE));
-      // Move it back to pending_close
-      regionStates.updateRegionState(REGIONINFO, State.PENDING_CLOSE);
-
-      // Now fake the region closing successfully over on the regionserver; the
-      // regionserver will have set the region in CLOSED state.  This will
-      // trigger callback into AM. The below zk close call is from the RS close
-      // region handler duplicated here because its down deep in a private
-      // method hard to expose.
-      int versionid =
-        ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
-      assertNotSame(versionid, -1);
-      // AM is going to notice above CLOSED and queue up a new assign.  The
-      // assign will go to open the region in the new location set by the
-      // balancer.  The zk node will be OFFLINE waiting for regionserver to
-      // transition it through OPENING, OPENED.  Wait till we see the OFFLINE
-      // zk node before we proceed.
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-
-      // Get current versionid else will fail on transition from OFFLINE to OPENING below
-      versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
-      assertNotSame(-1, versionid);
-      // This uglyness below is what the openregionhandler on RS side does.
-      versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
-        SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, versionid);
-      assertNotSame(-1, versionid);
-      // Move znode from OPENING to OPENED as RS does on successful open.
-      versionid =
-        ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO, SERVERNAME_B, versionid);
-      assertNotSame(-1, versionid);
-      // Wait on the handler removing the OPENED znode.
-      while(regionStates.isRegionInTransition(REGIONINFO)) Threads.sleep(1);
-    } finally {
-      executor.shutdown();
-      am.shutdown();
-      // Clean up all znodes
-      ZKAssign.deleteAllNodes(this.watcher);
-    }
-  }
-
-  /**
-   * Run a simple server shutdown handler.
-   * @throws KeeperException
-   * @throws IOException
-   */
-  @Test
-  public void testShutdownHandler()
-      throws KeeperException, IOException, CoordinatedStateException, ServiceException {
-    // Create and startup an executor.  This is used by AssignmentManager
-    // handling zk callbacks.
-    ExecutorService executor = startupMasterExecutor("testShutdownHandler");
-
-    // Create an AM.
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
-        this.server, this.serverManager);
-    try {
-      processServerShutdownHandler(am, false);
-    } finally {
-      executor.shutdown();
-      am.shutdown();
-      // Clean up all znodes
-      ZKAssign.deleteAllNodes(this.watcher);
-    }
-  }
-
-  /**
-   * To test closed region handler to remove rit and delete corresponding znode
-   * if region in pending close or closing while processing shutdown of a region
-   * server.(HBASE-5927).
-   *
-   * @throws KeeperException
-   * @throws IOException
-   * @throws ServiceException
-   */
-  @Test
-  public void testSSHWhenDisableTableInProgress() throws KeeperException, IOException,
-    CoordinatedStateException, ServiceException {
-    testCaseWithPartiallyDisabledState(Table.State.DISABLING);
-    testCaseWithPartiallyDisabledState(Table.State.DISABLED);
-  }
-
-
-  /**
-   * To test if the split region is removed from RIT if the region was in SPLITTING state but the RS
-   * has actually completed the splitting in hbase:meta but went down. See HBASE-6070 and also HBASE-5806
-   *
-   * @throws KeeperException
-   * @throws IOException
-   */
-  @Test
-  public void testSSHWhenSplitRegionInProgress() throws KeeperException, IOException, Exception {
-    // true indicates the region is split but still in RIT
-    testCaseWithSplitRegionPartial(true);
-    // false indicate the region is not split
-    testCaseWithSplitRegionPartial(false);
-  }
-
-  private void testCaseWithSplitRegionPartial(boolean regionSplitDone) throws KeeperException,
-      IOException, InterruptedException,
-    CoordinatedStateException, ServiceException {
-    // Create and startup an executor. This is used by AssignmentManager
-    // handling zk callbacks.
-    ExecutorService executor = startupMasterExecutor("testSSHWhenSplitRegionInProgress");
-    // We need a mocked catalog tracker.
-    ZKAssign.deleteAllNodes(this.watcher);
-
-    // Create an AM.
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
-      this.server, this.serverManager);
-    // adding region to regions and servers maps.
-    am.regionOnline(REGIONINFO, SERVERNAME_A);
-    // adding region in pending close.
-    am.getRegionStates().updateRegionState(
-      REGIONINFO, State.SPLITTING, SERVERNAME_A);
-    am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-      Table.State.ENABLED);
-    RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
-        REGIONINFO.getRegionName(), SERVERNAME_A);
-    String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
-    // create znode in M_ZK_REGION_CLOSING state.
-    ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
-
-    try {
-      processServerShutdownHandler(am, regionSplitDone);
-      // check znode deleted or not.
-      // In both cases the znode should be deleted.
-
-      if (regionSplitDone) {
-        assertFalse("Region state of region in SPLITTING should be removed from rit.",
-            am.getRegionStates().isRegionsInTransition());
-      } else {
-        while (!am.assignInvoked) {
-          Thread.sleep(1);
-        }
-        assertTrue("Assign should be invoked.", am.assignInvoked);
-      }
-    } finally {
-      REGIONINFO.setOffline(false);
-      REGIONINFO.setSplit(false);
-      executor.shutdown();
-      am.shutdown();
-      // Clean up all znodes
-      ZKAssign.deleteAllNodes(this.watcher);
-    }
-  }
-
-  private void testCaseWithPartiallyDisabledState(Table.State state) throws KeeperException,
-      IOException, CoordinatedStateException, ServiceException {
-    // Create and startup an executor. This is used by AssignmentManager
-    // handling zk callbacks.
-    ExecutorService executor = startupMasterExecutor("testSSHWhenDisableTableInProgress");
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
-    ZKAssign.deleteAllNodes(this.watcher);
-
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, executor, null, master.getTableLockManager());
-    // adding region to regions and servers maps.
-    am.regionOnline(REGIONINFO, SERVERNAME_A);
-    // adding region in pending close.
-    am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
-    if (state == Table.State.DISABLING) {
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.DISABLING);
-    } else {
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.DISABLED);
-    }
-    RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
-        REGIONINFO.getRegionName(), SERVERNAME_A);
-    // RegionTransitionData data = new
-    // RegionTransitionData(EventType.M_ZK_REGION_CLOSING,
-    // REGIONINFO.getRegionName(), SERVERNAME_A);
-    String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
-    // create znode in M_ZK_REGION_CLOSING state.
-    ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
-
-    try {
-      processServerShutdownHandler(am, false);
-      // check znode deleted or not.
-      // In both cases the znode should be deleted.
-      assertTrue("The znode should be deleted.", ZKUtil.checkExists(this.watcher, node) == -1);
-      // check whether in rit or not. In the DISABLING case also the below
-      // assert will be true but the piece of code added for HBASE-5927 will not
-      // do that.
-      if (state == Table.State.DISABLED) {
-        assertFalse("Region state of region in pending close should be removed from rit.",
-            am.getRegionStates().isRegionsInTransition());
-      }
-    } finally {
-      am.setEnabledTable(REGIONINFO.getTable());
-      executor.shutdown();
-      am.shutdown();
-      // Clean up all znodes
-      ZKAssign.deleteAllNodes(this.watcher);
-    }
-  }
-
-  private void processServerShutdownHandler(AssignmentManager am, boolean splitRegion)
-      throws IOException, ServiceException {
-    // Make sure our new AM gets callbacks; once registered, can't unregister.
-    // Thats ok because we make a new zk watcher for each test.
-    this.watcher.registerListenerFirst(am);
-
-    // Need to set up a fake scan of meta for the servershutdown handler
-    // Make an RS Interface implementation.  Make it so a scanner can go against it.
-    ClientProtos.ClientService.BlockingInterface implementation =
-      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-    // Get a meta row result that has region up on SERVERNAME_A
-
-    Result r;
-    if (splitRegion) {
-      r = MetaMockingUtil.getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
-    } else {
-      r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
-    }
-
-    final ScanResponse.Builder builder = ScanResponse.newBuilder();
-    builder.setMoreResults(true);
-    builder.addCellsPerResult(r.size());
-    final List<CellScannable> cellScannables = new ArrayList<CellScannable>(1);
-    cellScannables.add(r);
-    Mockito.when(implementation.scan(
-      (RpcController)Mockito.any(), (ScanRequest)Mockito.any())).
-      thenAnswer(new Answer<ScanResponse>() {
-          @Override
-          public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
-            PayloadCarryingRpcController controller = (PayloadCarryingRpcController) invocation
-                .getArguments()[0];
-            if (controller != null) {
-              controller.setCellScanner(CellUtil.createCellScanner(cellScannables));
-            }
-            return builder.build();
-          }
-      });
-
-    // Get a connection w/ mocked up common methods.
-    HConnection connection =
-      HConnectionTestingUtility.getMockedConnectionAndDecorate(HTU.getConfiguration(),
-        null, implementation, SERVERNAME_B, REGIONINFO);
-
-    // Make it so we can get a catalogtracker from servermanager.. .needed
-    // down in guts of server shutdown handler.
-    Mockito.when(this.server.getShortCircuitConnection()).thenReturn(connection);
-
-    // Now make a server shutdown handler instance and invoke process.
-    // Have it that SERVERNAME_A died.
-    DeadServer deadServers = new DeadServer();
-    deadServers.add(SERVERNAME_A);
-    // I need a services instance that will return the AM
-    MasterFileSystem fs = Mockito.mock(MasterFileSystem.class);
-    Mockito.doNothing().when(fs).setLogRecoveryMode();
-    Mockito.when(fs.getLogRecoveryMode()).thenReturn(RecoveryMode.LOG_REPLAY);
-    MasterServices services = Mockito.mock(MasterServices.class);
-    Mockito.when(services.getAssignmentManager()).thenReturn(am);
-    Mockito.when(services.getServerManager()).thenReturn(this.serverManager);
-    Mockito.when(services.getZooKeeper()).thenReturn(this.watcher);
-    Mockito.when(services.getMasterFileSystem()).thenReturn(fs);
-    ServerShutdownHandler handler = new ServerShutdownHandler(this.server,
-      services, deadServers, SERVERNAME_A, false);
-    am.failoverCleanupDone.set(true);
-    handler.process();
-    // The region in r will have been assigned.  It'll be up in zk as unassigned.
-  }
-
-  /**
-   * Create and startup executor pools. Start same set as master does (just
-   * run a few less).
-   * @param name Name to give our executor
-   * @return Created executor (be sure to call shutdown when done).
-   */
-  private ExecutorService startupMasterExecutor(final String name) {
-    // TODO: Move up into HBaseTestingUtility?  Generally useful.
-    ExecutorService executor = new ExecutorService(name);
-    executor.startExecutorService(ExecutorType.MASTER_OPEN_REGION, 3);
-    executor.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, 3);
-    executor.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, 3);
-    executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
-    return executor;
-  }
-
-  @Test
-  public void testUnassignWithSplitAtSameTime() throws KeeperException,
-      IOException, CoordinatedStateException {
-    // Region to use in test.
-    final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
-    // First amend the servermanager mock so that when we do send close of the
-    // first meta region on SERVERNAME_A, it will return true rather than
-    // default null.
-    Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, hri, -1)).thenReturn(true);
-    // Need a mocked catalog tracker.
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
-        .getConfiguration());
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, null, null, master.getTableLockManager());
-    try {
-      // First make sure my mock up basically works.  Unassign a region.
-      unassign(am, SERVERNAME_A, hri);
-      // This delete will fail if the previous unassign did wrong thing.
-      ZKAssign.deleteClosingNode(this.watcher, hri, SERVERNAME_A);
-      // Now put a SPLITTING region in the way.  I don't have to assert it
-      // go put in place.  This method puts it in place then asserts it still
-      // owns it by moving state from SPLITTING to SPLITTING.
-      int version = createNodeSplitting(this.watcher, hri, SERVERNAME_A);
-      // Now, retry the unassign with the SPLTTING in place.  It should just
-      // complete without fail; a sort of 'silent' recognition that the
-      // region to unassign has been split and no longer exists: TOOD: what if
-      // the split fails and the parent region comes back to life?
-      unassign(am, SERVERNAME_A, hri);
-      // This transition should fail if the znode has been messed with.
-      ZKAssign.transitionNode(this.watcher, hri, SERVERNAME_A,
-        EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
-      assertFalse(am.getRegionStates().isRegionInTransition(hri));
-    } finally {
-      am.shutdown();
-    }
-  }
-
-  /**
-   * Tests the processDeadServersAndRegionsInTransition should not fail with NPE
-   * when it failed to get the children. Let's abort the system in this
-   * situation
-   * @throws ServiceException
-   */
-  @Test(timeout = 60000)
-  public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
-      throws IOException, KeeperException, CoordinatedStateException,
-      InterruptedException, ServiceException {
-    final RecoverableZooKeeper recoverableZk = Mockito
-        .mock(RecoverableZooKeeper.class);
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
-      this.server, this.serverManager);
-    Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest",
-        null) {
-      @Override
-      public RecoverableZooKeeper getRecoverableZooKeeper() {
-        return recoverableZk;
-      }
-    };
-    ((ZooKeeperWatcher) zkw).registerListener(am);
-    Mockito.doThrow(new InterruptedException()).when(recoverableZk)
-        .getChildren("/hbase/region-in-transition", null);
-    am.setWatcher((ZooKeeperWatcher) zkw);
-    try {
-      am.processDeadServersAndRegionsInTransition(null);
-      fail("Expected to abort");
-    } catch (NullPointerException e) {
-      fail("Should not throw NPE");
-    } catch (RuntimeException e) {
-      assertEquals("Aborted", e.getLocalizedMessage());
-    }
-  }
-  /**
-   * TestCase verifies that the regionPlan is updated whenever a region fails to open
-   * and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546).
-   */
-  @Test(timeout = 60000)
-  public void testRegionPlanIsUpdatedWhenRegionFailsToOpen() throws IOException, KeeperException,
-      ServiceException, InterruptedException, CoordinatedStateException {
-    this.server.getConfiguration().setClass(
-      HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockedLoadBalancer.class,
-      LoadBalancer.class);
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
-      this.server, this.serverManager);
-    try {
-      // Boolean variable used for waiting until randomAssignment is called and
-      // new
-      // plan is generated.
-      AtomicBoolean gate = new AtomicBoolean(false);
-      if (balancer instanceof MockedLoadBalancer) {
-        ((MockedLoadBalancer) balancer).setGateVariable(gate);
-      }
-      ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
-      int v = ZKAssign.getVersion(this.watcher, REGIONINFO);
-      ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A,
-          EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_FAILED_OPEN, v);
-      String path = ZKAssign.getNodeName(this.watcher, REGIONINFO
-          .getEncodedName());
-      am.getRegionStates().updateRegionState(
-        REGIONINFO, State.OPENING, SERVERNAME_A);
-      // a dummy plan inserted into the regionPlans. This plan is cleared and
-      // new one is formed
-      am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan(
-          REGIONINFO, null, SERVERNAME_A));
-      RegionPlan regionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
-      List<ServerName> serverList = new ArrayList<ServerName>(2);
-      serverList.add(SERVERNAME_B);
-      Mockito.when(
-          this.serverManager.createDestinationServersList(SERVERNAME_A))
-          .thenReturn(serverList);
-      am.nodeDataChanged(path);
-      // here we are waiting until the random assignment in the load balancer is
-      // called.
-      while (!gate.get()) {
-        Thread.sleep(10);
-      }
-      // new region plan may take some time to get updated after random
-      // assignment is called and
-      // gate is set to true.
-      RegionPlan newRegionPlan = am.regionPlans
-          .get(REGIONINFO.getEncodedName());
-      while (newRegionPlan == null) {
-        Thread.sleep(10);
-        newRegionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
-      }
-      // the new region plan created may contain the same RS as destination but
-      // it should
-      // be new plan.
-      assertNotSame("Same region plan should not come", regionPlan,
-          newRegionPlan);
-      assertTrue("Destination servers should be different.", !(regionPlan
-          .getDestination().equals(newRegionPlan.getDestination())));
-
-      Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
-    } finally {
-      this.server.getConfiguration().setClass(
-          HConstants.HBASE_MASTER_LOADBALANCER_CLASS, SimpleLoadBalancer.class,
-          LoadBalancer.class);
-      am.getExecutorService().shutdown();
-      am.shutdown();
-    }
-  }
-
-  /**
-   * Mocked load balancer class used in the testcase to make sure that the testcase waits until
-   * random assignment is called and the gate variable is set to true.
-   */
-  public static class MockedLoadBalancer extends SimpleLoadBalancer {
-    private AtomicBoolean gate;
-
-    public void setGateVariable(AtomicBoolean gate) {
-      this.gate = gate;
-    }
-
-    @Override
-    public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers) {
-      ServerName randomServerName = super.randomAssignment(regionInfo, servers);
-      this.gate.set(true);
-      return randomServerName;
-    }
-
-    @Override
-    public Map<ServerName, List<HRegionInfo>> retainAssignment(
-        Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
-      this.gate.set(true);
-      return super.retainAssignment(regions, servers);
-    }
-  }
-
-  /**
-   * Test the scenario when the master is in failover and trying to process a
-   * region which is in Opening state on a dead RS. Master will force offline the
-   * region and put it in transition. AM relies on SSH to reassign it.
-   */
-  @Test(timeout = 60000)
-  public void testRegionInOpeningStateOnDeadRSWhileMasterFailover() throws IOException,
-      KeeperException, ServiceException, CoordinatedStateException, InterruptedException {
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
-      this.server, this.serverManager);
-    ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
-    int version = ZKAssign.getVersion(this.watcher, REGIONINFO);
-    ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, version);
-    RegionTransition rt = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_OPENING,
-        REGIONINFO.getRegionName(), SERVERNAME_A, HConstants.EMPTY_BYTE_ARRAY);
-    version = ZKAssign.getVersion(this.watcher, REGIONINFO);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(false);
-    am.getRegionStates().logSplit(SERVERNAME_A); // Assume log splitting is done
-    am.getRegionStates().createRegionState(REGIONINFO);
-    am.gate.set(false);
-
-    BaseCoordinatedStateManager cp = new ZkCoordinatedStateManager();
-    cp.initialize(server);
-    cp.start();
-
-    OpenRegionCoordination orc = cp.getOpenRegionCoordination();
-    ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
-      new ZkOpenRegionCoordination.ZkOpenRegionDetails();
-    zkOrd.setServerName(server.getServerName());
-    zkOrd.setVersion(version);
-
-    assertFalse(am.processRegionsInTransition(rt, REGIONINFO, orc, zkOrd));
-    am.getTableStateManager().setTableState(REGIONINFO.getTable(), Table.State.ENABLED);
-    processServerShutdownHandler(am, false);
-    // Waiting for the assignment to get completed.
-    while (!am.gate.get()) {
-      Thread.sleep(10);
-    }
-    assertTrue("The region should be assigned immediately.", null != am.regionPlans.get(REGIONINFO
-        .getEncodedName()));
-  }
-
-  /**
-   * Test verifies whether assignment is skipped for regions of tables in DISABLING state during
-   * clean cluster startup. See HBASE-6281.
-   *
-   * @throws KeeperException
-   * @throws IOException
-   * @throws Exception
-   */
-  @Test(timeout = 60000)
-  public void testDisablingTableRegionsAssignmentDuringCleanClusterStartup()
-      throws KeeperException, IOException, Exception {
-    this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
-        MockedLoadBalancer.class, LoadBalancer.class);
-    Mockito.when(this.serverManager.getOnlineServers()).thenReturn(
-        new HashMap<ServerName, ServerLoad>(0));
-    List<ServerName> destServers = new ArrayList<ServerName>(1);
-    destServers.add(SERVERNAME_A);
-    Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
-    // To avoid cast exception in DisableTableHandler process.
-    HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
-
-    CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      HTU.getConfiguration());
-    Server server = new HMaster(HTU.getConfiguration(), csm);
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
-        this.serverManager);
-
-    Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
-
-    // Make it so we can get a catalogtracker from servermanager.. .needed
-    // down in guts of server shutdown handler.
-    Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
-
-    AtomicBoolean gate = new AtomicBoolean(false);
-    if (balancer instanceof MockedLoadBalancer) {
-      ((MockedLoadBalancer) balancer).setGateVariable(gate);
-    }
-    try{
-      // set table in disabling state.
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.DISABLING);
-      am.joinCluster();
-      // should not call retainAssignment if we get empty regions in assignAllUserRegions.
-      assertFalse(
-          "Assign should not be invoked for disabling table regions during clean cluster startup.",
-          gate.get());
-      // need to change table state from disabling to disabled.
-      assertTrue("Table should be disabled.",
-          am.getTableStateManager().isTableState(REGIONINFO.getTable(),
-            Table.State.DISABLED));
-    } finally {
-      this.server.getConfiguration().setClass(
-        HConstants.HBASE_MASTER_LOADBALANCER_CLASS, SimpleLoadBalancer.class,
-        LoadBalancer.class);
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.ENABLED);
-      am.shutdown();
-    }
-  }
-
-  /**
-   * Test verifies whether all the enabling table regions assigned only once during master startup.
-   *
-   * @throws KeeperException
-   * @throws IOException
-   * @throws Exception
-   */
-  @Test
-  public void testMasterRestartWhenTableInEnabling() throws KeeperException, IOException, Exception {
-    enabling = true;
-    List<ServerName> destServers = new ArrayList<ServerName>(1);
-    destServers.add(SERVERNAME_A);
-    Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
-    HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
-    CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      HTU.getConfiguration());
-    Server server = new HMaster(HTU.getConfiguration(), csm);
-    Whitebox.setInternalState(server, "serverManager", this.serverManager);
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
-        this.serverManager);
-
-    Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
-
-    // Make it so we can get a catalogtracker from servermanager.. .needed
-    // down in guts of server shutdown handler.
-    Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
-
-    try {
-      // set table in enabling state.
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.ENABLING);
-      new EnableTableHandler(server, REGIONINFO.getTable(),
-          am, new NullTableLockManager(), true).prepare()
-          .process();
-      assertEquals("Number of assignments should be 1.", 1, assignmentCount);
-      assertTrue("Table should be enabled.",
-          am.getTableStateManager().isTableState(REGIONINFO.getTable(),
-            Table.State.ENABLED));
-    } finally {
-      enabling = false;
-      assignmentCount = 0;
-      am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-        Table.State.ENABLED);
-      am.shutdown();
-      ZKAssign.deleteAllNodes(this.watcher);
-    }
-  }
-
-  /**
-   * Test verifies whether stale znodes of unknown tables as for the hbase:meta will be removed or
-   * not.
-   * @throws KeeperException
-   * @throws IOException
-   * @throws Exception
-   */
-  @Test
-  public void testMasterRestartShouldRemoveStaleZnodesOfUnknownTableAsForMeta()
-      throws Exception {
-    List<ServerName> destServers = new ArrayList<ServerName>(1);
-    destServers.add(SERVERNAME_A);
-    Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
-    Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
-    HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
-    CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      HTU.getConfiguration());
-    Server server = new HMaster(HTU.getConfiguration(), csm);
-    Whitebox.setInternalState(server, "serverManager", this.serverManager);
-    AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
-        this.serverManager);
-
-    Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
-
-    // Make it so we can get a catalogtracker from servermanager.. .needed
-    // down in guts of server shutdown handler.
-    Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
-
-    try {
-      TableName tableName = TableName.valueOf("dummyTable");
-      // set table in enabling state.
-      am.getTableStateManager().setTableState(tableName,
-        Table.State.ENABLING);
-      am.joinCluster();
-      assertFalse("Table should not be present in zookeeper.",
-        am.getTableStateManager().isTablePresent(tableName));
-    } finally {
-    }
-  }
-  /**
-   * When a region is in transition, if the region server opening the region goes down,
-   * the region assignment takes a long time normally (waiting for timeout monitor to trigger assign).
-   * This test is to make sure SSH reassigns it right away.
-   */
-  @Test
-  public void testSSHTimesOutOpeningRegionTransition()
-      throws KeeperException, IOException, CoordinatedStateException, ServiceException {
-    // Create an AM.
-    AssignmentManagerWithExtrasForTesting am =
-      setUpMockedAssignmentManager(this.server, this.serverManager);
-    // adding region in pending open.
-    RegionState state = new RegionState(REGIONINFO,
-      State.OPENING, System.currentTimeMillis(), SERVERNAME_A);
-    am.getRegionStates().regionOnline(REGIONINFO, SERVERNAME_B);
-    am.getRegionStates().regionsInTransition.put(REGIONINFO.getEncodedName(), state);
-    // adding region plan
-    am.regionPlans.put(REGIONINFO.getEncodedName(),
-      new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
-    am.getTableStateManager().setTableState(REGIONINFO.getTable(),
-      Table.State.ENABLED);
-
-    try {
-      am.assignInvoked = false;
-      processServerShutdownHandler(am, false);
-      assertTrue(am.assignInvoked);
-    } finally {
-      am.getRegionStates().regionsInTransition.remove(REGIONINFO.getEncodedName());
-      am.regionPlans.remove(REGIONINFO.getEncodedName());
-    }
-  }
-
-  /**
-   * Scenario:<ul>
-   *  <li> master starts a close, and creates a znode</li>
-   *  <li> it fails just at this moment, before contacting the RS</li>
-   *  <li> while the second master is coming up, the targeted RS dies. But it's before ZK timeout so
-   *    we don't know, and we have an exception.</li>
-   *  <li> the master must handle this nicely and reassign.
-   *  </ul>
-   */
-  @Test
-  public void testClosingFailureDuringRecovery() throws Exception {
-
-    AssignmentManagerWithExtrasForTesting am =
-        setUpMockedAssignmentManager(this.server, this.serverManager);
-    ZKAssign.createNodeClosing(this.watcher, REGIONINFO, SERVERNAME_A);
-    am.getRegionStates().createRegionState(REGIONINFO);
-
-    assertFalse( am.getRegionStates().isRegionsInTransition() );
-
-    am.processRegionInTransition(REGIONINFO.getEncodedName(), REGIONINFO);
-
-    assertTrue( am.getRegionStates().isRegionsInTransition() );
-  }
-
-  /**
-   * Creates a new ephemeral node in the SPLITTING state for the specified region.
-   * Create it ephemeral in case regionserver dies mid-split.
-   *
-   * <p>Does not transition nodes from other states.  If a node already exists
-   * for this region, a {@link NodeExistsException} will be thrown.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as offline
-   * @param serverName server event originates from
-   * @return Version of znode created.
-   * @throws KeeperException
-   * @throws IOException
-   */
-  // Copied from SplitTransaction rather than open the method over there in
-  // the regionserver package.
-  private static int createNodeSplitting(final ZooKeeperWatcher zkw,
-      final HRegionInfo region, final ServerName serverName)
-  throws KeeperException, IOException {
-    RegionTransition rt =
-      RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
-        region.getRegionName(), serverName);
-
-    String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
-    if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, rt.toByteArray())) {
-      throw new IOException("Failed create of ephemeral " + node);
-    }
-    // Transition node from SPLITTING to SPLITTING and pick up version so we
-    // can be sure this znode is ours; version is needed deleting.
-    return transitionNodeSplitting(zkw, region, serverName, -1);
-  }
-
-  // Copied from SplitTransaction rather than open the method over there in
-  // the regionserver package.
-  private static int transitionNodeSplitting(final ZooKeeperWatcher zkw,
-      final HRegionInfo parent,
-      final ServerName serverName, final int version)
-  throws KeeperException, IOException {
-    return ZKAssign.transitionNode(zkw, parent, serverName,
-      EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
-  }
-
-  private void unassign(final AssignmentManager am, final ServerName sn,
-      final HRegionInfo hri) throws RegionException {
-    // Before I can unassign a region, I need to set it online.
-    am.regionOnline(hri, sn);
-    // Unassign region.
-    am.unassign(hri);
-  }
-
-  /**
-   * Create an {@link AssignmentManagerWithExtrasForTesting} that has mocked
-   * {@link CatalogTracker} etc.
-   * @param server
-   * @param manager
-   * @return An AssignmentManagerWithExtras with mock connections, etc.
-   * @throws IOException
-   * @throws KeeperException
-   */
-  private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
-      final ServerManager manager) throws IOException, KeeperException,
-        ServiceException, CoordinatedStateException {
-    // Make an RS Interface implementation. Make it so a scanner can go against
-    // it and a get to return the single region, REGIONINFO, this test is
-    // messing with. Needed when "new master" joins cluster. AM will try and
-    // rebuild its list of user regions and it will also get the HRI that goes
-    // with an encoded name by doing a Get on hbase:meta
-    ClientProtos.ClientService.BlockingInterface ri =
-      Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-    // Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
-    Result r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
-    final ScanResponse.Builder builder = ScanResponse.newBuilder();
-    builder.setMoreResults(true);
-    builder.addCellsPerResult(r.size());
-    final List<CellScannable> rows = new ArrayList<CellScannable>(1);
-    rows.add(r);
-    Answer<ScanResponse> ans = new Answer<ClientProtos.ScanResponse>() {
-      @Override
-      public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
-        PayloadCarryingRpcController controller = (PayloadCarryingRpcController) invocation
-            .getArguments()[0];
-        if (controller != null) {
-          controller.setCellScanner(CellUtil.createCellScanner(rows));
-        }
-        return builder.build();
-      }
-    };
-    if (enabling) {
-      Mockito.when(ri.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any()))
-          .thenAnswer(ans).thenAnswer(ans).thenAnswer(ans).thenAnswer(ans).thenAnswer(ans)
-          .thenReturn(ScanResponse.newBuilder().setMoreResults(false).build());
-    } else {
-      Mockito.when(ri.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any())).thenAnswer(
-          ans);
-    }
-    // If a get, return the above result too for REGIONINFO
-    GetResponse.Builder getBuilder = GetResponse.newBuilder();
-    getBuilder.setResult(ProtobufUtil.toResult(r));
-    Mockito.when(ri.get((RpcController)Mockito.any(), (GetRequest) Mockito.any())).
-      thenReturn(getBuilder.build());
-    // Get a connection w/ mocked up common methods.
-    HConnection connection = HConnectionTestingUtility.
-      getMockedConnectionAndDecorate(HTU.getConfiguration(), null,
-        ri, SERVERNAME_B, REGIONINFO);
-    // Make it so we can get the connection from our mocked catalogtracker
-    // Create and startup an executor. Used by AM handling zk callbacks.
-    ExecutorService executor = startupMasterExecutor("mockedAMExecutor");
-    this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
-    AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting(
-      server, connection, manager, this.balancer, executor, new NullTableLockManager());
-    return am;
-  }
-
-  /**
-   * An {@link AssignmentManager} with some extra facility used testing
-   */
-  class AssignmentManagerWithExtrasForTesting extends AssignmentManager {
-    // Keep a reference so can give it out below in {@link #getExecutorService}
-    private final ExecutorService es;
-    boolean processRITInvoked = false;
-    boolean assignInvoked = false;
-    AtomicBoolean gate = new AtomicBoolean(true);
-    private HConnection connection;
-
-    public AssignmentManagerWithExtrasForTesting(
-        final Server master, HConnection connection, final ServerManager serverManager,
-        final LoadBalancer balancer,
-        final ExecutorService service, final TableLockManager tableLockManager)
-            throws KeeperException, IOException, CoordinatedStateException {
-      super(master, serverManager, balancer, service, null, tableLockManager);
-      this.es = service;
-      this.connection = connection;
-    }
-
-    @Override
-    boolean processRegionInTransition(String encodedRegionName,
-        HRegionInfo regionInfo) throws KeeperException, IOException {
-      this.processRITInvoked = true;
-      return super.processRegionInTransition(encodedRegionName, regionInfo);
-    }
-
-    @Override
-    public void assign(HRegionInfo region, boolean setOfflineInZK, boolean forceNewPlan) {
-      if (enabling) {
-        assignmentCount++;
-        this.regionOnline(region, SERVERNAME_A);
-      } else {
-        super.assign(region, setOfflineInZK, forceNewPlan);
-        this.gate.set(true);
-      }
-    }
-
-    @Override
-    boolean assign(ServerName destination, List<HRegionInfo> regions)
-        throws InterruptedException {
-      if (enabling) {
-        for (HRegionInfo region : regions) {
-          assignmentCount++;
-          this.regionOnline(region, SERVERNAME_A);
-        }
-        return true;
-      }
-      return super.assign(destination, regions);
-    }
-
-    @Override
-    public void assign(List<HRegionInfo> regions)
-        throws IOException, InterruptedException {
-      assignInvoked = (regions != null && regions.size() > 0);
-      super.assign(regions);
-      this.gate.set(true);
-    }
-
-    /** reset the watcher */
-    void setWatcher(ZooKeeperWatcher watcher) {
-      this.watcher = watcher;
-    }
-
-    /**
-     * @return ExecutorService used by this instance.
-     */
-    ExecutorService getExecutorService() {
-      return this.es;
-    }
-
-    /*
-     * Convenient method to retrieve mocked up connection
-     */
-    HConnection getConnection() {
-      return this.connection;
-    }
-  }
-
-  /**
-   * Call joinCluster on the passed AssignmentManager.  Do it in a thread
-   * so it runs independent of what all else is going on.  Try to simulate
-   * an AM running insided a failed over master by clearing all in-memory
-   * AM state first.
-  */
-  private void startFakeFailedOverMasterAssignmentManager(final AssignmentManager am,
-      final ZooKeeperWatcher watcher) {
-    // Make sure our new AM gets callbacks; once registered, we can't unregister.
-    // Thats ok because we make a new zk watcher for each test.
-    watcher.registerListenerFirst(am);
-    Thread t = new Thread("RunAmJoinCluster") {
-      @Override
-      public void run() {
-        // Call the joinCluster function as though we were doing a master
-        // failover at this point. It will stall just before we go to add
-        // the RIT region to our RIT Map in AM at processRegionsInTransition.
-        // First clear any inmemory state from AM so it acts like a new master
-        // coming on line.
-        am.getRegionStates().regionsInTransition.clear();
-        am.regionPlans.clear();
-        try {
-          am.joinCluster();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        } catch (KeeperException e) {
-          throw new RuntimeException(e);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        } catch (CoordinatedStateException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-    t.start();
-    while (!t.isAlive()) Threads.sleep(1);
-  }
-
-  @Test
-  public void testForceAssignMergingRegion() throws Exception {
-    // Region to use in test.
-    final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
-    // Need a mocked catalog tracker.
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
-      server.getConfiguration());
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, null, null, master.getTableLockManager());
-    RegionStates regionStates = am.getRegionStates();
-    try {
-      // First set the state of the region to merging
-      regionStates.updateRegionState(hri, RegionState.State.MERGING);
-      // Now, try to assign it with force new plan
-      am.assign(hri, true, true);
-      assertEquals("The region should be still in merging state",
-        RegionState.State.MERGING, regionStates.getRegionState(hri).getState());
-    } finally {
-      am.shutdown();
-    }
-  }
-
-  /**
-   * Test assignment related ZK events are ignored by AM if the region is not known
-   * by AM to be in transition. During normal operation, all assignments are started
-   * by AM (not considering split/merge), if an event is received but the region
-   * is not in transition, the event must be a very late one. So it can be ignored.
-   * During master failover, since AM watches assignment znodes after failover cleanup
-   * is completed, when an event comes in, AM should already have the region in transition
-   * if ZK is used during the assignment action (only hbck doesn't use ZK for region
-   * assignment). So during master failover, we can ignored such events too.
-   */
-  @Test
-  public void testAssignmentEventIgnoredIfNotExpected() throws KeeperException, IOException,
-      CoordinatedStateException {
-    // Region to use in test.
-    final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
-      server.getConfiguration());
-    final AtomicBoolean zkEventProcessed = new AtomicBoolean(false);
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, null, null, master.getTableLockManager()) {
-
-      @Override
-      void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
-                        OpenRegionCoordination.OpenRegionDetails ord) {
-        super.handleRegion(rt, coordination, ord);
-        if (rt != null && Bytes.equals(hri.getRegionName(),
-          rt.getRegionName()) && rt.getEventType() == EventType.RS_ZK_REGION_OPENING) {
-          zkEventProcessed.set(true);
-        }
-      }
-    };
-    try {
-      // First make sure the region is not in transition
-      am.getRegionStates().regionOffline(hri);
-      zkEventProcessed.set(false); // Reset it before faking zk transition
-      this.watcher.registerListenerFirst(am);
-      assertFalse("The region should not be in transition",
-        am.getRegionStates().isRegionInTransition(hri));
-      ZKAssign.createNodeOffline(this.watcher, hri, SERVERNAME_A);
-      // Trigger a transition event
-      ZKAssign.transitionNodeOpening(this.watcher, hri, SERVERNAME_A);
-      long startTime = EnvironmentEdgeManager.currentTimeMillis();
-      while (!zkEventProcessed.get()) {
-        assertTrue("Timed out in waiting for ZK event to be processed",
-          EnvironmentEdgeManager.currentTimeMillis() - startTime < 30000);
-        Threads.sleepWithoutInterrupt(100);
-      }
-      assertFalse(am.getRegionStates().isRegionInTransition(hri));
-    } finally {
-      am.shutdown();
-    }
-  }
-
-  /**
-   * If a table is deleted, we should not be able to balance it anymore.
-   * Otherwise, the region will be brought back.
-   * @throws Exception
-   */
-  @Test
-  public void testBalanceRegionOfDeletedTable() throws Exception {
-    AssignmentManager am = new AssignmentManager(this.server, this.serverManager,
-      balancer, null, null, master.getTableLockManager());
-    RegionStates regionStates = am.getRegionStates();
-    HRegionInfo hri = REGIONINFO;
-    regionStates.createRegionState(hri);
-    assertFalse(regionStates.isRegionInTransition(hri));
-    RegionPlan plan = new RegionPlan(hri, SERVERNAME_A, SERVERNAME_B);
-    // Fake table is deleted
-    regionStates.tableDeleted(hri.getTable());
-    am.balance(plan);
-    assertFalse("The region should not in transition",
-      regionStates.isRegionInTransition(hri));
-  }
-
-  /**
-   * Tests an on-the-fly RPC that was scheduled for the earlier RS on the same port
-   * for openRegion. AM should assign this somewhere else. (HBASE-9721)
-   */
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testOpenCloseRegionRPCIntendedForPreviousServer() throws Exception {
-    Mockito.when(this.serverManager.sendRegionOpen(Mockito.eq(SERVERNAME_B), Mockito.eq(REGIONINFO),
-      Mockito.anyInt(), (List<ServerName>)Mockito.any()))
-      .thenThrow(new DoNotRetryIOException());
-    this.server.getConfiguration().setInt("hbase.assignment.maximum.attempts", 100);
-
-    HRegionInfo hri = REGIONINFO;
-    LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
-      server.getConfiguration());
-    // Create an AM.
-    AssignmentManager am = new AssignmentManager(this.server,
-      this.serverManager, balancer, null, null, master.getTableLockManager());
-    RegionStates regionStates = am.getRegionStates();
-    try {
-      am.regionPlans.put(REGIONINFO.getEncodedName(),
-        new RegionPlan(REGIONINFO, null, SERVERNAME_B));
-
-      // Should fail once, but succeed on the second attempt for the SERVERNAME_A
-      am.assign(hri, true, false);
-    } finally {
-      assertEquals(SERVERNAME_A, regionStates.getRegionState(REGIONINFO).getServerName());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/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 989b9f9..469a5d2 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.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -47,28 +48,22 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 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.ZooKeeperProtos;
 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.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -85,7 +80,8 @@ public class TestAssignmentManagerOnCluster {
   final static Configuration conf = TEST_UTIL.getConfiguration();
   private static Admin admin;
 
-  static void setupOnce() throws Exception {
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
     // Using the our load balancer to control region plans
     conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
       MyLoadBalancer.class, LoadBalancer.class);
@@ -98,13 +94,6 @@ 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();
@@ -187,11 +176,6 @@ public class TestAssignmentManagerOnCluster {
       am.addPlan(hri.getEncodedName(), plan);
       master.assignRegion(hri);
 
-      int version = ZKAssign.transitionNode(master.getZooKeeper(), hri,
-        destServer, EventType.M_ZK_REGION_OFFLINE,
-        EventType.RS_ZK_REGION_OPENING, 0);
-      assertEquals("TansitionNode should fail", -1, version);
-
       TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
@@ -402,7 +386,7 @@ public class TestAssignmentManagerOnCluster {
       // region is closing now, will be re-assigned automatically.
       // now, let's forcefully assign it again. it should be
       // assigned properly and no double-assignment
-      am.assign(hri, true, true);
+      am.assign(hri, true);
 
       // let's check if it's assigned after it's out of transition
       am.waitOnRegionToClearRegionsInTransition(hri);
@@ -572,20 +556,7 @@ public class TestAssignmentManagerOnCluster {
         }
       }
       am.regionOffline(hri);
-      ZooKeeperWatcher zkw = TEST_UTIL.getHBaseCluster().getMaster().getZooKeeper();
       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.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING);
       List<HRegionInfo> toAssignRegions = am.processServerShutdown(destServerName);
@@ -775,7 +746,7 @@ public class TestAssignmentManagerOnCluster {
       }
 
       // You can't assign a dead region before SSH
-      am.assign(hri, true, true);
+      am.assign(hri, true);
       RegionState state = regionStates.getRegionState(hri);
       assertTrue(state.isFailedClose());
 
@@ -833,7 +804,7 @@ public class TestAssignmentManagerOnCluster {
       assertTrue(regionStates.isRegionOffline(hri));
 
       // You can't assign a disabled region
-      am.assign(hri, true, true);
+      am.assign(hri, true);
       assertTrue(regionStates.isRegionOffline(hri));
 
       // You can't unassign a disabled region either
@@ -991,7 +962,7 @@ public class TestAssignmentManagerOnCluster {
       }
 
       // Wait till no more RIT, the region should be offline.
-      am.waitUntilNoRegionsInTransition(60000);
+      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       assertTrue(regionStates.isRegionOffline(hri));
     } finally {
       MyRegionServer.abortedServer = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index 87c34be..5735eaa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -95,13 +95,11 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -1505,10 +1503,8 @@ public class TestDistributedLogSplitting {
     return count;
   }
 
-  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
-  throws KeeperException, InterruptedException {
-    ZKAssign.blockUntilNoRIT(zkw);
-    master.assignmentManager.waitUntilNoRegionsInTransition(60000);
+  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
   }
 
   private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,

http://git-wip-us.apache.org/repos/asf/hbase/blob/17dff681/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index 6bfb618..dc45b26 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -18,28 +18,39 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
+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.MediumTests;
 import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-
-import java.io.IOException;
-import java.util.List;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import com.google.common.base.Joiner;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
+import com.google.common.base.Joiner;
 
 @Category(MediumTests.class)
 public class TestMaster {
@@ -55,7 +66,6 @@ public class TestMaster {
     // Start a cluster of two regionservers.
     TEST_UTIL.startMiniCluster(2);
     admin = TEST_UTIL.getHBaseAdmin();
-    TEST_UTIL.getHBaseCluster().getMaster().assignmentManager.initializeHandlerTrackers();
   }
 
   @AfterClass
@@ -64,6 +74,7 @@ public class TestMaster {
   }
 
   @Test
+  @SuppressWarnings("deprecation")
   public void testMasterOpsWhileSplitting() throws Exception {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     HMaster m = cluster.getMaster();