You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2018/10/17 17:53:34 UTC

hbase git commit: HBASE-21310 & HBASE-21311 Addendum fix failed UTs, some UTs are not present on branch-2.1 and some are a bit different in the implementation

Repository: hbase
Updated Branches:
  refs/heads/branch-2.1 47364d4db -> 46227c227


HBASE-21310 & HBASE-21311 Addendum fix failed UTs, some UTs are not present on branch-2.1 and some are a bit different in the implementation


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

Branch: refs/heads/branch-2.1
Commit: 46227c2275d6fecaacbebf83de1e5cf97880c5d2
Parents: 47364d4
Author: Duo Zhang <zh...@apache.org>
Authored: Wed Oct 17 18:41:32 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Oct 17 10:53:13 2018 -0700

----------------------------------------------------------------------
 ...tFromClientAfterSplittingRegionTestBase.java | 81 --------------------
 ...FromClientAfterSplittingRegionsTestBase.java |  8 --
 ...SnapshotFromClientAfterTruncateTestBase.java | 48 ------------
 ...hotFromClientGetCompactionStateTestBase.java | 44 -----------
 ...eSnapshotFromClientAfterSplittingRegion.java | 53 -------------
 ...eSnapshotFromClientAfterSplittingRegion.java | 67 ----------------
 ...bRestoreSnapshotFromClientAfterTruncate.java | 67 ----------------
 ...oreSnapshotFromClientGetCompactionState.java | 67 ----------------
 ...tRestoreSnapshotFromClientAfterTruncate.java | 53 -------------
 ...oreSnapshotFromClientGetCompactionState.java | 53 -------------
 10 files changed, 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/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
deleted file mode 100644
index d1108c0..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
+++ /dev/null
@@ -1,81 +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.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/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterSplittingRegionsTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterSplittingRegionsTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterSplittingRegionsTestBase.java
index 51fbb82..1b3b2e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterSplittingRegionsTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterSplittingRegionsTestBase.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.List;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
 public class RestoreSnapshotFromClientAfterSplittingRegionsTestBase
@@ -37,12 +35,6 @@ public class RestoreSnapshotFromClientAfterSplittingRegionsTestBase
     // Take a snapshot
     admin.snapshot(snapshotName1, tableName);
 
-    // Load more data
-    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
-
-    // Split the second region
-    splitRegion(regionInfos.get(1));
-
     // Restore the snapshot
     admin.disableTable(tableName);
     admin.restoreSnapshot(snapshotName1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterTruncateTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterTruncateTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterTruncateTestBase.java
deleted file mode 100644
index 11fd6f1..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientAfterTruncateTestBase.java
+++ /dev/null
@@ -1,48 +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.TableName;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.junit.Test;
-
-public class RestoreSnapshotFromClientAfterTruncateTestBase
-    extends RestoreSnapshotFromClientTestBase {
-
-  @Test
-  public void testRestoreSnapshotAfterTruncate() throws Exception {
-    TableName tableName = TableName.valueOf(getValidMethodName());
-    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
-    SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 500, FAMILY);
-    int numOfRows = 0;
-
-    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
-      numOfRows = countRows(table);
-    }
-    // take snapshot
-    admin.snapshot("snap", tableName);
-    admin.disableTable(tableName);
-    admin.truncateTable(tableName, false);
-    admin.disableTable(tableName);
-    admin.restoreSnapshot("snap");
-
-    admin.enableTable(tableName);
-    verifyRowCount(TEST_UTIL, tableName, numOfRows);
-    SnapshotTestingUtils.verifyReplicasCameOnline(tableName, admin, getNumReplicas());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientGetCompactionStateTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientGetCompactionStateTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientGetCompactionStateTestBase.java
deleted file mode 100644
index 03c908f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientGetCompactionStateTestBase.java
+++ /dev/null
@@ -1,44 +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 org.junit.Test;
-
-public class RestoreSnapshotFromClientGetCompactionStateTestBase
-    extends RestoreSnapshotFromClientTestBase {
-
-  @Test
-  public void testGetCompactionStateAfterRestoringSnapshot()
-      throws IOException, InterruptedException {
-    // Take a snapshot
-    admin.snapshot(snapshotName1, tableName);
-
-    // Restore the snapshot
-    admin.disableTable(tableName);
-    admin.restoreSnapshot(snapshotName1);
-
-    // Get the compaction state of the restored table
-    CompactionState compactionState = admin.getCompactionState(tableName);
-
-    // The compactionState should be NONE because the table is disabled
-    assertEquals(CompactionState.NONE, compactionState);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/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
deleted file mode 100644
index 65513f8..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientAfterSplittingRegion.java
+++ /dev/null
@@ -1,53 +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 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/46227c22/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
deleted file mode 100644
index a9e2b54..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientAfterSplittingRegion.java
+++ /dev/null
@@ -1,67 +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 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/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientAfterTruncate.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientAfterTruncate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientAfterTruncate.java
deleted file mode 100644
index 0abe83d..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientAfterTruncate.java
+++ /dev/null
@@ -1,67 +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 java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
-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 TestMobRestoreSnapshotFromClientAfterTruncate
-    extends RestoreSnapshotFromClientAfterTruncateTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestMobRestoreSnapshotFromClientAfterTruncate.class);
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    setupConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
-  }
-
-  protected static void setupConf(Configuration conf) {
-    RestoreSnapshotFromClientTestBase.setupConf(conf);
-    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
-  }
-
-  @Override
-  protected void createTable() throws Exception {
-    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
-  }
-
-  @Override
-  protected void verifyRowCount(HBaseTestingUtility util, TableName tableName, long expectedRows)
-      throws IOException {
-    MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
-  }
-
-  @Override
-  protected int countRows(Table table, byte[]... families) throws IOException {
-    return MobSnapshotTestingUtils.countMobRows(table, families);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientGetCompactionState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientGetCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientGetCompactionState.java
deleted file mode 100644
index c4d6ec3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobRestoreSnapshotFromClientGetCompactionState.java
+++ /dev/null
@@ -1,67 +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 java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
-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 TestMobRestoreSnapshotFromClientGetCompactionState
-    extends RestoreSnapshotFromClientGetCompactionStateTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestMobRestoreSnapshotFromClientGetCompactionState.class);
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    setupConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
-  }
-
-  protected static void setupConf(Configuration conf) {
-    RestoreSnapshotFromClientTestBase.setupConf(conf);
-    TEST_UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
-  }
-
-  @Override
-  protected void createTable() throws Exception {
-    MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, getNumReplicas(), FAMILY);
-  }
-
-  @Override
-  protected void verifyRowCount(HBaseTestingUtility util, TableName tableName, long expectedRows)
-      throws IOException {
-    MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
-  }
-
-  @Override
-  protected int countRows(Table table, byte[]... families) throws IOException {
-    return MobSnapshotTestingUtils.countMobRows(table, families);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientAfterTruncate.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientAfterTruncate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientAfterTruncate.java
deleted file mode 100644
index 6ce0fb3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientAfterTruncate.java
+++ /dev/null
@@ -1,53 +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 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 TestRestoreSnapshotFromClientAfterTruncate
-    extends RestoreSnapshotFromClientAfterTruncateTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestRestoreSnapshotFromClientAfterTruncate.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/46227c22/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientGetCompactionState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientGetCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientGetCompactionState.java
deleted file mode 100644
index 85eba79..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientGetCompactionState.java
+++ /dev/null
@@ -1,53 +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 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 TestRestoreSnapshotFromClientGetCompactionState
-    extends RestoreSnapshotFromClientGetCompactionStateTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestRestoreSnapshotFromClientGetCompactionState.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;
-  }
-}