You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/10/15 06:00:58 UTC

hbase git commit: HBASE-21310 Split TestCloneSnapshotFromClient

Repository: hbase
Updated Branches:
  refs/heads/master dde336f6e -> 6781918ca


HBASE-21310 Split TestCloneSnapshotFromClient


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

Branch: refs/heads/master
Commit: 6781918ca8e8a845f7e84d190f7a137a19a3a941
Parents: dde336f
Author: zhangduo <zh...@apache.org>
Authored: Mon Oct 15 08:34:15 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Mon Oct 15 14:00:40 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/TestCheckTestClasses.java      |   2 -
 ...tFromClientAfterSplittingRegionTestBase.java |  81 +++++
 ...FromClientCloneLinksAfterDeleteTestBase.java |  85 +++++
 .../CloneSnapshotFromClientErrorTestBase.java   |  41 +++
 .../CloneSnapshotFromClientNormalTestBase.java  |  61 ++++
 .../client/CloneSnapshotFromClientTestBase.java | 161 +++++++++
 .../client/TestCloneSnapshotFromClient.java     | 329 -------------------
 ...eSnapshotFromClientAfterSplittingRegion.java |  53 +++
 ...SnapshotFromClientCloneLinksAfterDelete.java |  54 +++
 .../TestCloneSnapshotFromClientError.java       |  52 +++
 .../TestCloneSnapshotFromClientNormal.java      |  52 +++
 ...oneSnapshotFromClientWithRegionReplicas.java |  38 ---
 .../client/TestMobCloneSnapshotFromClient.java  | 188 -----------
 ...eSnapshotFromClientAfterSplittingRegion.java |  67 ++++
 ...SnapshotFromClientCloneLinksAfterDelete.java | 131 ++++++++
 .../TestMobCloneSnapshotFromClientError.java    |  66 ++++
 .../TestMobCloneSnapshotFromClientNormal.java   |  74 +++++
 .../hbase/snapshot/MobSnapshotTestingUtils.java |  41 ++-
 18 files changed, 1001 insertions(+), 575 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
index a20ef73..d6b68fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
new file mode 100644
index 0000000..d1108c0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
@@ -0,0 +1,81 @@
+/**
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+public class CloneSnapshotFromClientAfterSplittingRegionTestBase
+    extends CloneSnapshotFromClientTestBase {
+
+  private void splitRegion(final RegionInfo regionInfo) throws IOException {
+    byte[][] splitPoints = Bytes.split(regionInfo.getStartKey(), regionInfo.getEndKey(), 1);
+    admin.split(regionInfo.getTable(), splitPoints[1]);
+  }
+
+  @Test
+  public void testCloneSnapshotAfterSplittingRegion() throws IOException, InterruptedException {
+    // Turn off the CatalogJanitor
+    admin.catalogJanitorSwitch(false);
+
+    try {
+      List<RegionInfo> regionInfos = admin.getRegions(tableName);
+      RegionReplicaUtil.removeNonDefaultRegions(regionInfos);
+
+      // Split the first region
+      splitRegion(regionInfos.get(0));
+
+      // Take a snapshot
+      admin.snapshot(snapshotName2, tableName);
+
+      // Clone the snapshot to another table
+      TableName clonedTableName =
+        TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
+      admin.cloneSnapshot(snapshotName2, clonedTableName);
+      SnapshotTestingUtils.waitForTableToBeOnline(TEST_UTIL, clonedTableName);
+
+      RegionStates regionStates =
+        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+
+      // The region count of the cloned table should be the same as the one of the original table
+      int openRegionCountOfOriginalTable =
+        regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OPEN).size();
+      int openRegionCountOfClonedTable =
+        regionStates.getRegionByStateOfTable(clonedTableName).get(RegionState.State.OPEN).size();
+      assertEquals(openRegionCountOfOriginalTable, openRegionCountOfClonedTable);
+
+      int splitRegionCountOfOriginalTable =
+        regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.SPLIT).size();
+      int splitRegionCountOfClonedTable =
+        regionStates.getRegionByStateOfTable(clonedTableName).get(RegionState.State.SPLIT).size();
+      assertEquals(splitRegionCountOfOriginalTable, splitRegionCountOfClonedTable);
+
+      TEST_UTIL.deleteTable(clonedTableName);
+    } finally {
+      admin.catalogJanitorSwitch(true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientCloneLinksAfterDeleteTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientCloneLinksAfterDeleteTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientCloneLinksAfterDeleteTestBase.java
new file mode 100644
index 0000000..254aeac
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientCloneLinksAfterDeleteTestBase.java
@@ -0,0 +1,85 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.TableName;
+import org.junit.Test;
+
+public class CloneSnapshotFromClientCloneLinksAfterDeleteTestBase
+    extends CloneSnapshotFromClientTestBase {
+
+  /**
+   * Verify that tables created from the snapshot are still alive after source table deletion.
+   */
+  @Test
+  public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
+    // Clone a table from the first snapshot
+    final TableName clonedTableName =
+      TableName.valueOf(getValidMethodName() + "1-" + System.currentTimeMillis());
+    admin.cloneSnapshot(snapshotName0, clonedTableName);
+    verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
+
+    // Take a snapshot of this cloned table.
+    admin.disableTable(clonedTableName);
+    admin.snapshot(snapshotName2, clonedTableName);
+
+    // Clone the snapshot of the cloned table
+    final TableName clonedTableName2 =
+      TableName.valueOf(getValidMethodName() + "2-" + System.currentTimeMillis());
+    admin.cloneSnapshot(snapshotName2, clonedTableName2);
+    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
+    admin.disableTable(clonedTableName2);
+
+    // Remove the original table
+    TEST_UTIL.deleteTable(tableName);
+    waitCleanerRun();
+
+    // Verify the first cloned table
+    admin.enableTable(clonedTableName);
+    verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
+
+    // Verify the second cloned table
+    admin.enableTable(clonedTableName2);
+    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
+    admin.disableTable(clonedTableName2);
+
+    // Delete the first cloned table
+    TEST_UTIL.deleteTable(clonedTableName);
+    waitCleanerRun();
+
+    // Verify the second cloned table
+    admin.enableTable(clonedTableName2);
+    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
+
+    // Clone a new table from cloned
+    final TableName clonedTableName3 =
+      TableName.valueOf(getValidMethodName() + "3-" + System.currentTimeMillis());
+    admin.cloneSnapshot(snapshotName2, clonedTableName3);
+    verifyRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
+
+    // Delete the cloned tables
+    TEST_UTIL.deleteTable(clonedTableName2);
+    TEST_UTIL.deleteTable(clonedTableName3);
+    admin.deleteSnapshot(snapshotName2);
+  }
+
+  private void waitCleanerRun() throws InterruptedException {
+    TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientErrorTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientErrorTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientErrorTestBase.java
new file mode 100644
index 0000000..04df8e4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientErrorTestBase.java
@@ -0,0 +1,41 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
+import org.junit.Test;
+
+public class CloneSnapshotFromClientErrorTestBase extends CloneSnapshotFromClientTestBase {
+
+  @Test(expected = SnapshotDoesNotExistException.class)
+  public void testCloneNonExistentSnapshot() throws IOException, InterruptedException {
+    String snapshotName = "random-snapshot-" + System.currentTimeMillis();
+    final TableName tableName =
+      TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
+    admin.cloneSnapshot(snapshotName, tableName);
+  }
+
+  @Test(expected = NamespaceNotFoundException.class)
+  public void testCloneOnMissingNamespace() throws IOException, InterruptedException {
+    final TableName clonedTableName = TableName.valueOf("unknownNS:" + getValidMethodName());
+    admin.cloneSnapshot(snapshotName1, clonedTableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
new file mode 100644
index 0000000..3e26076
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
@@ -0,0 +1,61 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.junit.Test;
+
+public class CloneSnapshotFromClientNormalTestBase extends CloneSnapshotFromClientTestBase {
+
+  @Test
+  public void testCloneSnapshot() throws IOException, InterruptedException {
+    TableName clonedTableName =
+      TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
+    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
+    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
+    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
+  }
+
+  private void testCloneSnapshot(TableName tableName, byte[] snapshotName, int snapshotRows)
+      throws IOException, InterruptedException {
+    // create a new table from snapshot
+    admin.cloneSnapshot(snapshotName, tableName);
+    verifyRowCount(TEST_UTIL, tableName, snapshotRows);
+
+    verifyReplicasCameOnline(tableName);
+    TEST_UTIL.deleteTable(tableName);
+  }
+
+  private void verifyReplicasCameOnline(TableName tableName) throws IOException {
+    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
+  }
+
+  @Test
+  public void testCloneSnapshotCrossNamespace() throws IOException, InterruptedException {
+    String nsName = getValidMethodName() + "_ns_" + System.currentTimeMillis();
+    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    final TableName clonedTableName =
+      TableName.valueOf(nsName, getValidMethodName() + "-" + System.currentTimeMillis());
+    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
+    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
+    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
new file mode 100644
index 0000000..c74ec1d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
@@ -0,0 +1,161 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+/**
+ * Base class for testing clone snapsot
+ */
+public class CloneSnapshotFromClientTestBase {
+
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  protected final byte[] FAMILY = Bytes.toBytes("cf");
+
+  protected byte[] emptySnapshot;
+  protected byte[] snapshotName0;
+  protected byte[] snapshotName1;
+  protected byte[] snapshotName2;
+  protected TableName tableName;
+  protected int snapshot0Rows;
+  protected int snapshot1Rows;
+  protected Admin admin;
+
+  @Rule
+  public TestName name = new TestName();
+
+  protected static void setupConfiguration() {
+    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
+    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    setupConfiguration();
+    TEST_UTIL.startMiniCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  protected final String getValidMethodName() {
+    return name.getMethodName().replaceAll("[^0-9A-Za-z_]", "_");
+  }
+
+  /**
+   * Initialize the tests with a table filled with some data and two snapshots (snapshotName0,
+   * snapshotName1) of different states. The tableName, snapshotNames and the number of rows in the
+   * snapshot are initialized.
+   */
+  @Before
+  public void setup() throws Exception {
+    this.admin = TEST_UTIL.getAdmin();
+
+    long tid = System.currentTimeMillis();
+    tableName = TableName.valueOf(getValidMethodName() + tid);
+    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
+    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
+    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
+    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+
+    createTableAndSnapshots();
+  }
+
+  protected void createTable() throws IOException, InterruptedException {
+    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
+  }
+
+  protected int numRowsToLoad() {
+    return 500;
+  }
+
+  protected int countRows(Table table) throws IOException {
+    return TEST_UTIL.countRows(table);
+  }
+
+  private void createTableAndSnapshots() throws Exception {
+    // create Table and disable it
+    createTable();
+    admin.disableTable(tableName);
+
+    // take an empty snapshot
+    admin.snapshot(emptySnapshot, tableName);
+
+    // enable table and insert data
+    admin.enableTable(tableName);
+    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, numRowsToLoad(), FAMILY);
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      snapshot0Rows = countRows(table);
+    }
+    admin.disableTable(tableName);
+
+    // take a snapshot
+    admin.snapshot(snapshotName0, tableName);
+
+    // enable table and insert more data
+    admin.enableTable(tableName);
+    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, numRowsToLoad(), FAMILY);
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      snapshot1Rows = countRows(table);
+    }
+    admin.disableTable(tableName);
+
+    // take a snapshot of the updated table
+    admin.snapshot(snapshotName1, tableName);
+
+    // re-enable table
+    admin.enableTable(tableName);
+  }
+
+  protected int getNumReplicas() {
+    return 1;
+  }
+
+  protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
+      long expectedRows) throws IOException {
+    SnapshotTestingUtils.verifyRowCount(util, tableName, expectedRows);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (admin.tableExists(tableName)) {
+      TEST_UTIL.deleteTable(tableName);
+    }
+    SnapshotTestingUtils.deleteAllSnapshots(admin);
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
deleted file mode 100644
index 157b18f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
+++ /dev/null
@@ -1,329 +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.client;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
-import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test clone snapshots from the client
- */
-@Category({LargeTests.class, ClientTests.class})
-public class TestCloneSnapshotFromClient {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestCloneSnapshotFromClient.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestCloneSnapshotFromClient.class);
-
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  protected final byte[] FAMILY = Bytes.toBytes("cf");
-
-  protected byte[] emptySnapshot;
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
-  protected TableName tableName;
-  protected int snapshot0Rows;
-  protected int snapshot1Rows;
-  protected Admin admin;
-
-  @Rule
-  public TestName name = new TestName();
-
-  protected static void setupConfiguration() {
-    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
-    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
-    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
-    TEST_UTIL.getConfiguration().setBoolean(
-        "hbase.master.enabletable.roundrobin", true);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    setupConfiguration();
-    TEST_UTIL.startMiniCluster(3);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Initialize the tests with a table filled with some data
-   * and two snapshots (snapshotName0, snapshotName1) of different states.
-   * The tableName, snapshotNames and the number of rows in the snapshot are initialized.
-   */
-  @Before
-  public void setup() throws Exception {
-    this.admin = TEST_UTIL.getAdmin();
-
-    long tid = System.currentTimeMillis();
-    tableName = TableName.valueOf(name.getMethodName() + tid);
-    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
-
-    createTableAndSnapshots();
-  }
-
-  protected void createTableAndSnapshots() throws Exception {
-    // create Table and disable it
-    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
-    admin.disableTable(tableName);
-
-    // take an empty snapshot
-    admin.snapshot(emptySnapshot, tableName);
-
-    // enable table and insert data
-    admin.enableTable(tableName);
-    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
-    try (Table table = TEST_UTIL.getConnection().getTable(tableName)){
-      snapshot0Rows = TEST_UTIL.countRows(table);
-    }
-    admin.disableTable(tableName);
-
-    // take a snapshot
-    admin.snapshot(snapshotName0, tableName);
-
-    // enable table and insert more data
-    admin.enableTable(tableName);
-    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
-    try (Table table = TEST_UTIL.getConnection().getTable(tableName)){
-      snapshot1Rows = TEST_UTIL.countRows(table);
-    }
-    admin.disableTable(tableName);
-
-    // take a snapshot of the updated table
-    admin.snapshot(snapshotName1, tableName);
-
-    // re-enable table
-    admin.enableTable(tableName);
-  }
-
-  protected int getNumReplicas() {
-    return 1;
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (admin.tableExists(tableName)) {
-      TEST_UTIL.deleteTable(tableName);
-    }
-    SnapshotTestingUtils.deleteAllSnapshots(admin);
-    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
-  }
-
-  @Test(expected=SnapshotDoesNotExistException.class)
-  public void testCloneNonExistentSnapshot() throws IOException, InterruptedException {
-    String snapshotName = "random-snapshot-" + System.currentTimeMillis();
-    final TableName tableName = TableName.valueOf(name.getMethodName() + "-"
-      + System.currentTimeMillis());
-    admin.cloneSnapshot(snapshotName, tableName);
-  }
-
-  @Test(expected = NamespaceNotFoundException.class)
-  public void testCloneOnMissingNamespace() throws IOException, InterruptedException {
-    final TableName clonedTableName = TableName.valueOf("unknownNS:" + name.getMethodName());
-    admin.cloneSnapshot(snapshotName1, clonedTableName);
-  }
-
-  @Test
-  public void testCloneSnapshot() throws IOException, InterruptedException {
-    final TableName clonedTableName = TableName.valueOf(name.getMethodName() + "-"
-      + System.currentTimeMillis());
-    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
-    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
-    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
-  }
-
-  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
-      int snapshotRows) throws IOException, InterruptedException {
-    // create a new table from snapshot
-    admin.cloneSnapshot(snapshotName, tableName);
-    verifyRowCount(TEST_UTIL, tableName, snapshotRows);
-
-    verifyReplicasCameOnline(tableName);
-    TEST_UTIL.deleteTable(tableName);
-  }
-
-  protected void verifyReplicasCameOnline(TableName tableName) throws IOException {
-    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
-  }
-
-  @Test
-  public void testCloneSnapshotCrossNamespace() throws IOException, InterruptedException {
-    String nsName = "testCloneSnapshotCrossNamespace";
-    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
-    final TableName clonedTableName = TableName.valueOf(nsName, name.getMethodName()
-      + "-" + System.currentTimeMillis());
-    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
-    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
-    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
-  }
-
-  /**
-   * Verify that tables created from the snapshot are still alive after source table deletion.
-   */
-  @Test
-  public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
-    // Clone a table from the first snapshot
-    final TableName clonedTableName = TableName.valueOf(name.getMethodName() + "1-"
-      + System.currentTimeMillis());
-    admin.cloneSnapshot(snapshotName0, clonedTableName);
-    verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
-
-    // Take a snapshot of this cloned table.
-    admin.disableTable(clonedTableName);
-    admin.snapshot(snapshotName2, clonedTableName);
-
-    // Clone the snapshot of the cloned table
-    final TableName clonedTableName2 = TableName.valueOf(name.getMethodName() + "2-"
-      + System.currentTimeMillis());
-    admin.cloneSnapshot(snapshotName2, clonedTableName2);
-    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
-    admin.disableTable(clonedTableName2);
-
-    // Remove the original table
-    TEST_UTIL.deleteTable(tableName);
-    waitCleanerRun();
-
-    // Verify the first cloned table
-    admin.enableTable(clonedTableName);
-    verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
-
-    // Verify the second cloned table
-    admin.enableTable(clonedTableName2);
-    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
-    admin.disableTable(clonedTableName2);
-
-    // Delete the first cloned table
-    TEST_UTIL.deleteTable(clonedTableName);
-    waitCleanerRun();
-
-    // Verify the second cloned table
-    admin.enableTable(clonedTableName2);
-    verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
-
-    // Clone a new table from cloned
-    final TableName clonedTableName3 = TableName.valueOf(name.getMethodName() + "3-"
-      + System.currentTimeMillis());
-    admin.cloneSnapshot(snapshotName2, clonedTableName3);
-    verifyRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
-
-    // Delete the cloned tables
-    TEST_UTIL.deleteTable(clonedTableName2);
-    TEST_UTIL.deleteTable(clonedTableName3);
-    admin.deleteSnapshot(snapshotName2);
-  }
-
-  @Test
-  public void testCloneSnapshotAfterSplittingRegion() throws IOException, InterruptedException {
-    // Turn off the CatalogJanitor
-    admin.catalogJanitorSwitch(false);
-
-    try {
-      List<RegionInfo> regionInfos = admin.getRegions(tableName);
-      RegionReplicaUtil.removeNonDefaultRegions(regionInfos);
-
-      // Split the first region
-      splitRegion(regionInfos.get(0));
-
-      // Take a snapshot
-      admin.snapshot(snapshotName2, tableName);
-
-      // Clone the snapshot to another table
-      TableName clonedTableName = TableName.valueOf(name.getMethodName() + "-"
-        + System.currentTimeMillis());
-      admin.cloneSnapshot(snapshotName2, clonedTableName);
-      SnapshotTestingUtils.waitForTableToBeOnline(TEST_UTIL, clonedTableName);
-
-      RegionStates regionStates =
-        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
-
-      // The region count of the cloned table should be the same as the one of the original table
-      int openRegionCountOfOriginalTable =
-        regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OPEN).size();
-      int openRegionCountOfClonedTable =
-        regionStates.getRegionByStateOfTable(clonedTableName).get(RegionState.State.OPEN).size();
-      assertEquals(openRegionCountOfOriginalTable, openRegionCountOfClonedTable);
-
-      int splitRegionCountOfOriginalTable =
-        regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.SPLIT).size();
-      int splitRegionCountOfClonedTable =
-        regionStates.getRegionByStateOfTable(clonedTableName).get(RegionState.State.SPLIT).size();
-      assertEquals(splitRegionCountOfOriginalTable, splitRegionCountOfClonedTable);
-
-      TEST_UTIL.deleteTable(clonedTableName);
-    } finally {
-      admin.catalogJanitorSwitch(true);
-    }
-  }
-
-  // ==========================================================================
-  //  Helpers
-  // ==========================================================================
-
-  private void waitCleanerRun() throws InterruptedException {
-    TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
-  }
-
-  protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
-      long expectedRows) throws IOException {
-    SnapshotTestingUtils.verifyRowCount(util, tableName, expectedRows);
-  }
-
-  protected void splitRegion(final RegionInfo regionInfo) throws IOException {
-    byte[][] splitPoints = Bytes.split(regionInfo.getStartKey(), regionInfo.getEndKey(), 1);
-    admin.split(regionInfo.getTable(), splitPoints[1]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientAfterSplittingRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientAfterSplittingRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientAfterSplittingRegion.java
new file mode 100644
index 0000000..65513f8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientAfterSplittingRegion.java
@@ -0,0 +1,53 @@
+/**
+ * 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.client;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
+public class TestCloneSnapshotFromClientAfterSplittingRegion
+    extends CloneSnapshotFromClientAfterSplittingRegionTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloneSnapshotFromClientAfterSplittingRegion.class);
+
+  @Parameter
+  public int numReplicas;
+
+  @Parameters(name = "{index}: regionReplication={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { 1 }, new Object[] { 3 });
+  }
+
+  @Override
+  protected int getNumReplicas() {
+    return numReplicas;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCloneLinksAfterDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCloneLinksAfterDelete.java
new file mode 100644
index 0000000..e5f1fad
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCloneLinksAfterDelete.java
@@ -0,0 +1,54 @@
+/**
+ * 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.client;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
+public class TestCloneSnapshotFromClientCloneLinksAfterDelete
+    extends CloneSnapshotFromClientCloneLinksAfterDeleteTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloneSnapshotFromClientCloneLinksAfterDelete.class);
+
+  @Parameter
+  public int numReplicas;
+
+  @Parameters(name = "{index}: regionReplication={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { 1 }, new Object[] { 3 });
+  }
+
+  @Override
+  protected int getNumReplicas() {
+    return numReplicas;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientError.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientError.java
new file mode 100644
index 0000000..93c2388
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientError.java
@@ -0,0 +1,52 @@
+/**
+ * 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.client;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
+public class TestCloneSnapshotFromClientError extends CloneSnapshotFromClientErrorTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloneSnapshotFromClientError.class);
+
+  @Parameter
+  public int numReplicas;
+
+  @Parameters(name = "{index}: regionReplication={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { 1 }, new Object[] { 3 });
+  }
+
+  @Override
+  protected int getNumReplicas() {
+    return numReplicas;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientNormal.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientNormal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientNormal.java
new file mode 100644
index 0000000..c7730d0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientNormal.java
@@ -0,0 +1,52 @@
+/**
+ * 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.client;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
+public class TestCloneSnapshotFromClientNormal extends CloneSnapshotFromClientNormalTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloneSnapshotFromClientNormal.class);
+
+  @Parameter
+  public int numReplicas;
+
+  @Parameters(name = "{index}: regionReplication={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { 1 }, new Object[] { 3 });
+  }
+
+  @Override
+  protected int getNumReplicas() {
+    return numReplicas;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java
deleted file mode 100644
index b67c288..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientWithRegionReplicas.java
+++ /dev/null
@@ -1,38 +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.client;
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.ClassRule;
-import org.junit.experimental.categories.Category;
-
-@Category({LargeTests.class, ClientTests.class})
-public class TestCloneSnapshotFromClientWithRegionReplicas extends
-    TestCloneSnapshotFromClient {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestCloneSnapshotFromClientWithRegionReplicas.class);
-
-  @Override
-  protected int getNumReplicas() {
-    return 3;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
deleted file mode 100644
index 069ec7a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClient.java
+++ /dev/null
@@ -1,188 +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.client;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Optional;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
-import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-/**
- * Test clone snapshots from the client
- */
-@Category({LargeTests.class, ClientTests.class})
-public class TestMobCloneSnapshotFromClient extends TestCloneSnapshotFromClient {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClient.class);
-
-  private static boolean delayFlush = false;
-
-  @Rule
-  public TestName name = new TestName();
-
-  protected static void setupConfiguration() {
-    TestCloneSnapshotFromClient.setupConfiguration();
-    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
-    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    setupConfiguration();
-    TEST_UTIL.startMiniCluster(3);
-  }
-
-  @Override
-  protected void createTableAndSnapshots() throws Exception {
-    // create Table and disable it
-    createMobTable(TEST_UTIL, tableName, SnapshotTestingUtils.getSplitKeys(), getNumReplicas(),
-      FAMILY);
-    delayFlush = false;
-    admin.disableTable(tableName);
-
-    // take an empty snapshot
-    admin.snapshot(emptySnapshot, tableName);
-
-    Connection c = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-    Table table = c.getTable(tableName);
-    try {
-      // enable table and insert data
-      admin.enableTable(tableName);
-      SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
-      snapshot0Rows = MobSnapshotTestingUtils.countMobRows(table);
-      admin.disableTable(tableName);
-
-      // take a snapshot
-      admin.snapshot(snapshotName0, tableName);
-
-      // enable table and insert more data
-      admin.enableTable(tableName);
-      SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
-      snapshot1Rows = MobSnapshotTestingUtils.countMobRows(table);
-      admin.disableTable(tableName);
-
-      // take a snapshot of the updated table
-      admin.snapshot(snapshotName1, tableName);
-
-      // re-enable table
-      admin.enableTable(tableName);
-    } finally {
-      table.close();
-    }
-  }
-
-  @Test
-  @Override
-  public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
-    // delay the flush to make sure
-    delayFlush = true;
-    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
-    long tid = System.currentTimeMillis();
-    byte[] snapshotName3 = Bytes.toBytes("snaptb3-" + tid);
-    TableName clonedTableName3 = TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
-    admin.snapshot(snapshotName3, tableName);
-    delayFlush = false;
-    int snapshot3Rows = -1;
-    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
-      snapshot3Rows = TEST_UTIL.countRows(table);
-    }
-    admin.cloneSnapshot(snapshotName3, clonedTableName3);
-    admin.deleteSnapshot(snapshotName3);
-    super.testCloneLinksAfterDelete();
-    verifyRowCount(TEST_UTIL, clonedTableName3, snapshot3Rows);
-    admin.disableTable(clonedTableName3);
-    admin.deleteTable(clonedTableName3);
-  }
-
-  @Override
-  protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
-      long expectedRows) throws IOException {
-    MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
-  }
-
-  /**
-   * This coprocessor is used to delay the flush.
-   */
-  public static class DelayFlushCoprocessor implements RegionCoprocessor, RegionObserver {
-
-    @Override
-    public Optional<RegionObserver> getRegionObserver() {
-      return Optional.of(this);
-    }
-
-    @Override
-    public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e,
-        FlushLifeCycleTracker tracker) throws IOException {
-      if (delayFlush) {
-        try {
-          if (Bytes.compareTo(e.getEnvironment().getRegionInfo().getStartKey(),
-            HConstants.EMPTY_START_ROW) != 0) {
-            Thread.sleep(100);
-          }
-        } catch (InterruptedException e1) {
-          throw new InterruptedIOException(e1.getMessage());
-        }
-      }
-    }
-  }
-
-  private void createMobTable(final HBaseTestingUtility util, final TableName tableName,
-    final byte[][] splitKeys, int regionReplication, final byte[]... families) throws IOException,
-    InterruptedException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.setRegionReplication(regionReplication);
-    htd.addCoprocessor(DelayFlushCoprocessor.class.getName());
-    for (byte[] family : families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      hcd.setMobEnabled(true);
-      hcd.setMobThreshold(0L);
-      htd.addFamily(hcd);
-    }
-    util.getAdmin().createTable(htd, splitKeys);
-    SnapshotTestingUtils.waitForTableToBeOnline(util, tableName);
-    assertEquals((splitKeys.length + 1) * regionReplication,
-      util.getAdmin().getTableRegions(tableName).size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientAfterSplittingRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientAfterSplittingRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientAfterSplittingRegion.java
new file mode 100644
index 0000000..a9e2b54
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientAfterSplittingRegion.java
@@ -0,0 +1,67 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestMobCloneSnapshotFromClientAfterSplittingRegion
+    extends CloneSnapshotFromClientAfterSplittingRegionTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClientAfterSplittingRegion.class);
+
+  protected static void setupConfiguration() {
+    CloneSnapshotFromClientTestBase.setupConfiguration();
+    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    setupConfiguration();
+    TEST_UTIL.startMiniCluster(3);
+  }
+
+  @Override
+  protected void createTable() throws IOException, InterruptedException {
+    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName,
+      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), FAMILY);
+  }
+
+  @Override
+  protected int numRowsToLoad() {
+    return 20;
+  }
+
+  @Override
+  protected int countRows(Table table) throws IOException {
+    return MobSnapshotTestingUtils.countMobRows(table);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
new file mode 100644
index 0000000..1cae7a0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
@@ -0,0 +1,131 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestMobCloneSnapshotFromClientCloneLinksAfterDelete
+    extends CloneSnapshotFromClientCloneLinksAfterDeleteTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClientCloneLinksAfterDelete.class);
+
+  private static boolean delayFlush = false;
+
+  /**
+   * This coprocessor is used to delay the flush.
+   */
+  public static class DelayFlushCoprocessor implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e,
+        FlushLifeCycleTracker tracker) throws IOException {
+      if (delayFlush) {
+        try {
+          if (Bytes.compareTo(e.getEnvironment().getRegionInfo().getStartKey(),
+            HConstants.EMPTY_START_ROW) != 0) {
+            Thread.sleep(100);
+          }
+        } catch (InterruptedException e1) {
+          throw new InterruptedIOException(e1.getMessage());
+        }
+      }
+    }
+  }
+
+  protected static void setupConfiguration() {
+    CloneSnapshotFromClientTestBase.setupConfiguration();
+    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    setupConfiguration();
+    TEST_UTIL.startMiniCluster(3);
+  }
+
+  @Override
+  protected void createTable() throws IOException, InterruptedException {
+    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName,
+      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), DelayFlushCoprocessor.class.getName(),
+      FAMILY);
+  }
+
+  @Override
+  protected int numRowsToLoad() {
+    return 20;
+  }
+
+  @Override
+  protected int countRows(Table table) throws IOException {
+    return MobSnapshotTestingUtils.countMobRows(table);
+  }
+
+  @Test
+  @Override
+  public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
+    // delay the flush to make sure
+    delayFlush = true;
+    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
+    long tid = System.currentTimeMillis();
+    byte[] snapshotName3 = Bytes.toBytes("snaptb3-" + tid);
+    TableName clonedTableName3 =
+      TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
+    admin.snapshot(snapshotName3, tableName);
+    delayFlush = false;
+    int snapshot3Rows = -1;
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      snapshot3Rows = TEST_UTIL.countRows(table);
+    }
+    admin.cloneSnapshot(snapshotName3, clonedTableName3);
+    admin.deleteSnapshot(snapshotName3);
+    super.testCloneLinksAfterDelete();
+    verifyRowCount(TEST_UTIL, clonedTableName3, snapshot3Rows);
+    admin.disableTable(clonedTableName3);
+    admin.deleteTable(clonedTableName3);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientError.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientError.java
new file mode 100644
index 0000000..b691e60
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientError.java
@@ -0,0 +1,66 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestMobCloneSnapshotFromClientError extends CloneSnapshotFromClientErrorTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClientError.class);
+
+  protected static void setupConfiguration() {
+    CloneSnapshotFromClientTestBase.setupConfiguration();
+    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    setupConfiguration();
+    TEST_UTIL.startMiniCluster(3);
+  }
+
+  @Override
+  protected void createTable() throws IOException, InterruptedException {
+    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName,
+      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), FAMILY);
+  }
+
+  @Override
+  protected int numRowsToLoad() {
+    return 20;
+  }
+
+  @Override
+  protected int countRows(Table table) throws IOException {
+    return MobSnapshotTestingUtils.countMobRows(table);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientNormal.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientNormal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientNormal.java
new file mode 100644
index 0000000..94b938e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientNormal.java
@@ -0,0 +1,74 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestMobCloneSnapshotFromClientNormal extends CloneSnapshotFromClientNormalTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMobCloneSnapshotFromClientNormal.class);
+
+  protected static void setupConfiguration() {
+    CloneSnapshotFromClientTestBase.setupConfiguration();
+    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    setupConfiguration();
+    TEST_UTIL.startMiniCluster(3);
+  }
+
+  @Override
+  protected void createTable() throws IOException, InterruptedException {
+    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName,
+      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), FAMILY);
+  }
+
+  @Override
+  protected int numRowsToLoad() {
+    return 20;
+  }
+
+  @Override
+  protected int countRows(Table table) throws IOException {
+    return MobSnapshotTestingUtils.countMobRows(table);
+  }
+
+  @Override
+  protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
+      long expectedRows) throws IOException {
+    MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6781918c/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index 75b6c23..9ee4336 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@ -20,10 +20,14 @@ package org.apache.hadoop.hbase.snapshot;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 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.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
@@ -32,11 +36,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 
@@ -59,23 +59,28 @@ public class MobSnapshotTestingUtils {
       1, families);
   }
 
-  private static void createMobTable(final HBaseTestingUtility util,
-      final TableName tableName, final byte[][] splitKeys, int regionReplication,
-      final byte[]... families) throws IOException, InterruptedException {
-    TableDescriptorBuilder builder
-      = TableDescriptorBuilder.newBuilder(tableName)
-            .setRegionReplication(regionReplication);
+  public static void createMobTable(final HBaseTestingUtility util, final TableName tableName,
+      final byte[][] splitKeys, int regionReplication, final byte[]... families)
+      throws IOException, InterruptedException {
+    createMobTable(util, tableName, splitKeys, regionReplication, null, families);
+  }
+
+  public static void createMobTable(HBaseTestingUtility util, TableName tableName,
+      byte[][] splitKeys, int regionReplication, String cpClassName, byte[]... families)
+      throws IOException, InterruptedException {
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication);
     for (byte[] family : families) {
-      builder.setColumnFamily(ColumnFamilyDescriptorBuilder
-          .newBuilder(family)
-          .setMobEnabled(true)
-          .setMobThreshold(0L)
-          .build());
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+        .setMobThreshold(0L).build());
+    }
+    if (!StringUtils.isBlank(cpClassName)) {
+      builder.setCoprocessor(cpClassName);
     }
     util.getAdmin().createTable(builder.build(), splitKeys);
     SnapshotTestingUtils.waitForTableToBeOnline(util, tableName);
-    assertEquals((splitKeys.length + 1) * regionReplication, util
-        .getAdmin().getTableRegions(tableName).size());
+    assertEquals((splitKeys.length + 1) * regionReplication,
+      util.getAdmin().getRegions(tableName).size());
   }
 
   /**