HBASE-7559 Add additional Snapshots Unit Test Coverage (Aleksandr Shulman)

git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/hbase-7290@1445858 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-02-13 19:03:55 +00:00
parent 78a837f00a
commit 03d5cb8082
3 changed files with 906 additions and 56 deletions

View File

@ -0,0 +1,373 @@
/**
* 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.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
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.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test to verify that the cloned table is independent of the table from which it was cloned
*/
@Category(LargeTests.class)
public class TestSnapshotCloneIndependence {
private static final Log LOG = LogFactory.getLog(TestSnapshotCloneIndependence.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final int NUM_RS = 2;
private static final String STRING_TABLE_NAME = "test";
private static final String TEST_FAM_STR = "fam";
private static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
/**
* Setup the config for the cluster and start it
* @throws Exception on failure
*/
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(NUM_RS);
}
private static void setupConf(Configuration conf) {
// enable snapshot support
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
// disable the ui
conf.setInt("hbase.regionsever.info.port", -1);
// change the flush size to a small amount, regulating number of store files
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
// so make sure we get a compaction when doing a load, but keep around
// some files in the store
conf.setInt("hbase.hstore.compaction.min", 10);
conf.setInt("hbase.hstore.compactionThreshold", 10);
// block writes if we get to 12 store files
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.regionserver.msginterval", 100);
conf.setInt("hbase.client.pause", 250);
conf.setInt("hbase.client.retries.number", 6);
conf.setBoolean("hbase.master.enabletable.roundrobin", true);
}
@Before
public void setup() throws Exception {
UTIL.createTable(TABLE_NAME, TEST_FAM);
}
@After
public void tearDown() throws Exception {
UTIL.deleteTable(TABLE_NAME);
// and cleanup the archive directory
try {
UTIL.getTestFileSystem().delete(new Path(UTIL.getDefaultRootDirPath(), ".archive"), true);
} catch (IOException e) {
LOG.warn("Failure to delete archive directory", e);
}
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
/**
* Verify that adding data to the cloned table will not affect the original, and vice-versa when
* it is taken as an online snapshot.
*/
@Test
public void testOnlineSnapshotAppendIndependent() throws Exception {
runTestSnapshotAppendIndependent(true);
}
/**
* Verify that adding data to the cloned table will not affect the original, and vice-versa when
* it is taken as an offline snapshot.
*/
@Test
public void testOfflineSnapshotAppendIndependent() throws Exception {
runTestSnapshotAppendIndependent(false);
}
/**
* Verify that adding metadata to the cloned table will not affect the original, and vice-versa
* when it is taken as an online snapshot.
*/
@Test
public void testOnlineSnapshotMetadataChangesIndependent() throws Exception {
runTestSnapshotMetadataChangesIndependent(true);
}
/**
* Verify that adding netadata to the cloned table will not affect the original, and vice-versa
* when is taken as an online snapshot.
*/
@Test
public void testOfflineSnapshotMetadataChangesIndependent() throws Exception {
runTestSnapshotMetadataChangesIndependent(false);
}
/**
* Verify that region operations, in this case splitting a region, are independent between the
* cloned table and the original.
*/
@Test
public void testOfflineSnapshotRegionOperationsIndependent() throws Exception {
runTestRegionOperationsIndependent(false);
}
/**
* Verify that region operations, in this case splitting a region, are independent between the
* cloned table and the original.
*/
@Test
public void testOnlineSnapshotRegionOperationsIndependent() throws Exception {
runTestRegionOperationsIndependent(true);
}
private static void waitOnSplit(final HTable t, int originalCount) throws Exception {
for (int i = 0; i < 45; i++) {
try {
Thread.sleep(50);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
}
if (t.getRegionLocations().size() > originalCount) {
return;
}
}
throw new Exception("Split did not increase the number of regions");
}
/*
* Take a snapshot of a table, add data, and verify that this only
* affects one table
* @param online - Whether the table is online or not during the snapshot
*/
private void runTestSnapshotAppendIndependent(boolean online) throws Exception {
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
HBaseAdmin admin = UTIL.getHBaseAdmin();
final long startTime = System.currentTimeMillis();
final String localTableNameAsString = STRING_TABLE_NAME + startTime;
HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
try {
UTIL.loadTable(original, TEST_FAM);
final int origTableRowCount = UTIL.countRows(original);
// Take a snapshot
final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
snapshotNameAsString, rootDir, fs, online);
if (!online) {
admin.enableTable(localTableNameAsString);
}
byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
admin.cloneSnapshot(snapshotName, cloneTableName);
HTable clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName);
try {
final int clonedTableRowCount = UTIL.countRows(clonedTable);
Assert.assertEquals(
"The line counts of original and cloned tables do not match after clone. ",
origTableRowCount, clonedTableRowCount);
// Attempt to add data to the test
final String rowKey = "new-row-" + System.currentTimeMillis();
Put p = new Put(Bytes.toBytes(rowKey));
p.add(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
original.put(p);
original.flushCommits();
// Verify that it is not present in the original table
Assert.assertEquals("The row count of the original table was not modified by the put",
origTableRowCount + 1, UTIL.countRows(original));
Assert.assertEquals(
"The row count of the cloned table changed as a result of addition to the original",
clonedTableRowCount, UTIL.countRows(clonedTable));
p = new Put(Bytes.toBytes(rowKey));
p.add(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
clonedTable.put(p);
clonedTable.flushCommits();
// Verify that the new family is not in the restored table's description
Assert.assertEquals(
"The row count of the original table was modified by the put to the clone",
origTableRowCount + 1, UTIL.countRows(original));
Assert.assertEquals("The row count of the cloned table was not modified by the put",
clonedTableRowCount + 1, UTIL.countRows(clonedTable));
} finally {
clonedTable.close();
}
} finally {
original.close();
}
}
/*
* Take a snapshot of a table, do a split, and verify that this only affects one table
* @param online - Whether the table is online or not during the snapshot
*/
private void runTestRegionOperationsIndependent(boolean online) throws Exception {
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
// Create a table
HBaseAdmin admin = UTIL.getHBaseAdmin();
final long startTime = System.currentTimeMillis();
final String localTableNameAsString = STRING_TABLE_NAME + startTime;
HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
UTIL.loadTable(original, TEST_FAM);
final int loadedTableCount = UTIL.countRows(original);
System.out.println("Original table has: " + loadedTableCount + " rows");
final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
// Create a snapshot
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
snapshotNameAsString, rootDir, fs, online);
if (!online) {
admin.enableTable(localTableNameAsString);
}
byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
// Clone the snapshot
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
admin.cloneSnapshot(snapshotName, cloneTableName);
// Verify that region information is the same pre-split
original.clearRegionCache();
List<HRegionInfo> originalTableHRegions = admin.getTableRegions(Bytes
.toBytes(localTableNameAsString));
final int originalRegionCount = originalTableHRegions.size();
final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
Assert.assertEquals(
"The number of regions in the cloned table is different than in the original table.",
originalRegionCount, cloneTableRegionCount);
// Split a region on the parent table
admin.split(originalTableHRegions.get(0).getRegionName());
waitOnSplit(original, originalRegionCount);
// Verify that the cloned table region is not split
final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
Assert.assertEquals(
"The number of regions in the cloned table changed though none of its regions were split.",
cloneTableRegionCount, cloneTableRegionCount2);
}
/*
* Take a snapshot of a table, add metadata, and verify that this only
* affects one table
* @param online - Whether the table is online or not during the snapshot
*/
private void runTestSnapshotMetadataChangesIndependent(boolean online) throws Exception {
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
// Create a table
HBaseAdmin admin = UTIL.getHBaseAdmin();
final long startTime = System.currentTimeMillis();
final String localTableNameAsString = STRING_TABLE_NAME + startTime;
HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
UTIL.loadTable(original, TEST_FAM);
final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
// Create a snapshot
SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
snapshotNameAsString, rootDir, fs, online);
if (!online) {
admin.enableTable(localTableNameAsString);
}
byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
// Clone the snapshot
byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
admin.cloneSnapshot(snapshotName, cloneTableName);
// Add a new column family to the original table
byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
admin.disableTable(localTableNameAsString);
admin.addColumn(localTableNameAsString, hcd);
// Verify that it is not in the snapshot
admin.enableTable(localTableNameAsString);
// get a description of the cloned table
// get a list of its families
// assert that the family is there
HTableDescriptor originalTableDescriptor = original.getTableDescriptor();
HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
Assert.assertTrue("The original family was not found. There is something wrong. ",
originalTableDescriptor.hasFamily(TEST_FAM));
Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
clonedTableDescriptor.hasFamily(TEST_FAM));
Assert.assertTrue("The new family was not found. ",
originalTableDescriptor.hasFamily(TEST_FAM_2));
Assert.assertTrue("The new family was not found. ",
!clonedTableDescriptor.hasFamily(TEST_FAM_2));
}
}

View File

@ -0,0 +1,294 @@
/**
* 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.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class to verify that metadata is consistent before and after a snapshot attempt.
*/
@Category(MediumTests.class)
public class TestSnapshotMetadata {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final int NUM_RS = 2;
private static final String STRING_TABLE_NAME = "testtable";
private static final String MAX_VERSIONS_FAM_STR = "fam_max_columns";
private static final byte[] MAX_VERSIONS_FAM = Bytes.toBytes(MAX_VERSIONS_FAM_STR);
private static final String COMPRESSED_FAM_STR = "fam_compressed";
private static final byte[] COMPRESSED_FAM = Bytes.toBytes(COMPRESSED_FAM_STR);
private static final String BLOCKSIZE_FAM_STR = "fam_blocksize";
private static final byte[] BLOCKSIZE_FAM = Bytes.toBytes(BLOCKSIZE_FAM_STR);
private static final String BLOOMFILTER_FAM_STR = "fam_bloomfilter";
private static final byte[] BLOOMFILTER_FAM = Bytes.toBytes(BLOOMFILTER_FAM_STR);
byte[][] families = { MAX_VERSIONS_FAM, BLOOMFILTER_FAM, COMPRESSED_FAM, BLOCKSIZE_FAM };
private static final DataBlockEncoding DATA_BLOCK_ENCODING_TYPE = DataBlockEncoding.FAST_DIFF;
private static final BloomType BLOOM_TYPE = BloomType.ROW;
private static final int BLOCK_SIZE = 98;
private static final int MAX_VERSIONS = 8;
HBaseAdmin admin;
private String originalTableDescription;
private HTableDescriptor originalTableDescriptor;
byte[] originalTableName;
String originalTableNameAsString;
private static FileSystem fs;
private static Path rootDir;
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(NUM_RS);
fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
}
private static void setupConf(Configuration conf) {
// enable snapshot support
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
// disable the ui
conf.setInt("hbase.regionsever.info.port", -1);
// change the flush size to a small amount, regulating number of store files
conf.setInt("hbase.hregion.memstore.flush.size", 25000);
// so make sure we get a compaction when doing a load, but keep around
// some files in the store
conf.setInt("hbase.hstore.compaction.min", 10);
conf.setInt("hbase.hstore.compactionThreshold", 10);
// block writes if we get to 12 store files
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.regionserver.msginterval", 100);
conf.setInt("hbase.client.pause", 250);
conf.setInt("hbase.client.retries.number", 6);
conf.setBoolean("hbase.master.enabletable.roundrobin", true);
}
@Before
public void setup() throws Exception {
createTableWithNonDefaultProperties();
}
@After
public void tearDown() throws Exception {
admin.close();
}
/*
* Create a table that has non-default properties so we can see if they hold
*/
private void createTableWithNonDefaultProperties() throws Exception {
// create a table
admin = new HBaseAdmin(UTIL.getConfiguration());
final long startTime = System.currentTimeMillis();
final String sourceTableNameAsString = STRING_TABLE_NAME + startTime;
originalTableName = Bytes.toBytes(sourceTableNameAsString);
// enable replication on a column family
HColumnDescriptor maxVersionsColumn = new HColumnDescriptor(MAX_VERSIONS_FAM);
HColumnDescriptor bloomFilterColumn = new HColumnDescriptor(BLOOMFILTER_FAM);
HColumnDescriptor dataBlockColumn = new HColumnDescriptor(COMPRESSED_FAM);
HColumnDescriptor blockSizeColumn = new HColumnDescriptor(BLOCKSIZE_FAM);
maxVersionsColumn.setMaxVersions(MAX_VERSIONS);
bloomFilterColumn.setBloomFilterType(BLOOM_TYPE);
dataBlockColumn.setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE);
blockSizeColumn.setBlocksize(BLOCK_SIZE);
HTableDescriptor htd = new HTableDescriptor(sourceTableNameAsString);
htd.addFamily(maxVersionsColumn);
htd.addFamily(bloomFilterColumn);
htd.addFamily(dataBlockColumn);
htd.addFamily(blockSizeColumn);
admin.createTable(htd);
HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
originalTableNameAsString = sourceTableNameAsString;
originalTableDescriptor = original.getTableDescriptor();
originalTableDescription = originalTableDescriptor.toString();
original.close();
}
/**
* Verify that the describe for a cloned table matches the describe from the original.
*/
@Test
public void testDescribeMatchesAfterClone() throws Exception {
// Clone the original table
final String clonedTableNameAsString = "clone" + originalTableNameAsString;
final byte[] clonedTableName = Bytes.toBytes(clonedTableNameAsString);
final String snapshotNameAsString = "snapshot" + originalTableNameAsString
+ System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
// restore the snapshot into a cloned table and examine the output
List<byte[]> familiesList = new ArrayList<byte[]>();
for (byte[] family : families) {
familiesList.add(family);
}
// Create a snapshot in which all families are empty
SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableNameAsString, null,
familiesList, snapshotNameAsString, rootDir, fs);
admin.cloneSnapshot(snapshotName, clonedTableName);
HTable clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName);
Assert.assertEquals(
originalTableDescription.replace(originalTableNameAsString, clonedTableNameAsString),
clonedTable.getTableDescriptor().toString());
admin.enableTable(originalTableName);
clonedTable.close();
}
/**
* Verify that the describe for a restored table matches the describe for one the original.
*/
@Test
public void testDescribeMatchesAfterRestore() throws Exception {
runRestoreWithAdditionalMetadata(false);
}
/**
* Verify that if metadata changed after a snapshot was taken, that the old metadata replaces the
* new metadata during a restore
*/
@Test
public void testDescribeMatchesAfterMetadataChangeAndRestore() throws Exception {
runRestoreWithAdditionalMetadata(true);
}
/**
* Verify that when the table is empty, making metadata changes after the restore does not affect
* the restored table's original metadata
* @throws Exception
*/
@Test
public void testDescribeOnEmptyTableMatchesAfterMetadataChangeAndRestore() throws Exception {
runRestoreWithAdditionalMetadata(true, false);
}
private void runRestoreWithAdditionalMetadata(boolean changeMetadata) throws Exception {
runRestoreWithAdditionalMetadata(changeMetadata, true);
}
private void runRestoreWithAdditionalMetadata(boolean changeMetadata, boolean addData)
throws Exception {
if (admin.isTableDisabled(originalTableName)) {
admin.enableTable(originalTableName);
}
// populate it with data
final byte[] familyForUpdate = BLOCKSIZE_FAM;
List<byte[]> familiesWithDataList = new ArrayList<byte[]>();
List<byte[]> emptyFamiliesList = new ArrayList<byte[]>();
if (addData) {
HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
UTIL.loadTable(original, familyForUpdate); // family arbitrarily chosen
original.close();
for (byte[] family : families) {
if (family != familyForUpdate) {
emptyFamiliesList.add(family);
}
}
familiesWithDataList.add(familyForUpdate);
} else {
for (byte[] family : families) {
emptyFamiliesList.add(family);
}
}
// take a snapshot
final String snapshotNameAsString = "snapshot" + originalTableNameAsString
+ System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableNameAsString,
familiesWithDataList, emptyFamiliesList, snapshotNameAsString, rootDir, fs);
admin.enableTable(originalTableName);
if (changeMetadata) {
final String newFamilyNameAsString = "newFamily" + System.currentTimeMillis();
final byte[] newFamilyName = Bytes.toBytes(newFamilyNameAsString);
admin.disableTable(originalTableName);
HColumnDescriptor hcd = new HColumnDescriptor(newFamilyName);
admin.addColumn(originalTableName, hcd);
Assert.assertTrue("New column family was not added.",
admin.getTableDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
}
// restore it
if (!admin.isTableDisabled(originalTableName)) {
admin.disableTable(originalTableName);
}
admin.restoreSnapshot(snapshotName);
admin.enableTable(originalTableName);
HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
// verify that the descrption is reverted
try {
Assert
.assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
Assert.assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));
} finally {
original.close();
}
}
}

View File

@ -24,6 +24,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
@ -37,6 +38,7 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -60,27 +62,51 @@ public class SnapshotTestingUtils {
/**
* Assert that we don't have any snapshots lists
* @throws IOException if the admin operation fails
*
* @throws IOException
* if the admin operation fails
*/
public static void assertNoSnapshots(HBaseAdmin admin) throws IOException {
assertEquals("Have some previous snapshots", 0, admin.listSnapshots().size());
assertEquals("Have some previous snapshots", 0, admin.listSnapshots()
.size());
}
/**
* Make sure that there is only one snapshot returned from the master and its name and table match
* the passed in parameters.
* Make sure that there is only one snapshot returned from the master and its
* name and table match the passed in parameters.
*/
public static void assertOneSnapshotThatMatches(HBaseAdmin admin, SnapshotDescription snapshot)
public static List<SnapshotDescription> assertExistsMatchingSnapshot(
HBaseAdmin admin, String snapshotName, String tableName)
throws IOException {
// list the snapshot
List<SnapshotDescription> snapshots = admin.listSnapshots();
List<SnapshotDescription> returnedSnapshots = new ArrayList<SnapshotDescription>();
for (SnapshotDescription sd : snapshots) {
if (snapshotName.equals(sd.getName()) && tableName.equals(sd.getTable())) {
returnedSnapshots.add(sd);
}
}
Assert.assertTrue("No matching snapshots found.", returnedSnapshots.size()>0);
return returnedSnapshots;
}
/**
* Make sure that there is only one snapshot returned from the master
*/
public static void assertOneSnapshotThatMatches(HBaseAdmin admin,
SnapshotDescription snapshot) throws IOException {
assertOneSnapshotThatMatches(admin, snapshot.getName(), snapshot.getTable());
}
/**
* Make sure that there is only one snapshot returned from the master and its name and table match
* the passed in parameters.
* Make sure that there is only one snapshot returned from the master and its
* name and table match the passed in parameters.
*/
public static List<SnapshotDescription> assertOneSnapshotThatMatches(HBaseAdmin admin,
String snapshotName, String tableName) throws IOException {
public static List<SnapshotDescription> assertOneSnapshotThatMatches(
HBaseAdmin admin, String snapshotName, String tableName)
throws IOException {
// list the snapshot
List<SnapshotDescription> snapshots = admin.listSnapshots();
@ -92,33 +118,77 @@ public class SnapshotTestingUtils {
}
/**
* Make sure that there is only one snapshot returned from the master and its name and table match
* the passed in parameters.
* Make sure that there is only one snapshot returned from the master and its
* name and table match the passed in parameters.
*/
public static List<SnapshotDescription> assertOneSnapshotThatMatches(HBaseAdmin admin,
byte[] snapshot, byte[] tableName) throws IOException {
return assertOneSnapshotThatMatches(admin, Bytes.toString(snapshot), Bytes.toString(tableName));
public static List<SnapshotDescription> assertOneSnapshotThatMatches(
HBaseAdmin admin, byte[] snapshot, byte[] tableName) throws IOException {
return assertOneSnapshotThatMatches(admin, Bytes.toString(snapshot),
Bytes.toString(tableName));
}
/**
* Confirm that the snapshot contains references to all the files that should be in the snapshot
* Multi-family version of the confirmSnapshotValid function
*/
public static void confirmSnapshotValid(SnapshotDescription snapshotDescriptor, byte[] tableName,
byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs, boolean requireLogs,
public static void confirmSnapshotValid(
SnapshotDescription snapshotDescriptor, byte[] tableName,
List<byte[]> nonEmptyTestFamilies, List<byte[]> emptyTestFamilies,
Path rootDir, HBaseAdmin admin, FileSystem fs, boolean requireLogs,
Path logsDir, Set<String> snapshotServers) throws IOException {
Path snapshotDir = SnapshotDescriptionUtils
.getCompletedSnapshotDir(snapshotDescriptor, rootDir);
if (nonEmptyTestFamilies != null) {
for (byte[] testFamily : nonEmptyTestFamilies) {
confirmSnapshotValid(snapshotDescriptor, tableName, testFamily,
rootDir, admin, fs, requireLogs, logsDir, false, null);
}
}
if (emptyTestFamilies != null) {
for (byte[] testFamily : emptyTestFamilies) {
confirmSnapshotValid(snapshotDescriptor, tableName, testFamily,
rootDir, admin, fs, requireLogs, logsDir, true, null);
}
}
}
/**
* Confirm that the snapshot contains references to all the files that should
* be in the snapshot.
*/
public static void confirmSnapshotValid(
SnapshotDescription snapshotDescriptor, byte[] tableName,
byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
boolean requireLogs, Path logsDir, Set<String> snapshotServers)
throws IOException {
confirmSnapshotValid(snapshotDescriptor, tableName, testFamily, rootDir,
admin, fs, requireLogs, logsDir, false, snapshotServers);
}
/**
* Confirm that the snapshot contains references to all the files that should
* be in the snapshot.
*/
public static void confirmSnapshotValid(
SnapshotDescription snapshotDescriptor, byte[] tableName,
byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
boolean requireLogs, Path logsDir, boolean familyEmpty,
Set<String> snapshotServers) throws IOException {
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
snapshotDescriptor, rootDir);
assertTrue(fs.exists(snapshotDir));
Path snapshotinfo = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
Path snapshotinfo = new Path(snapshotDir,
SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
assertTrue(fs.exists(snapshotinfo));
// check the logs dir
if (requireLogs) {
TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logsDir, snapshotServers,
snapshotDescriptor, new Path(snapshotDir, HConstants.HREGION_LOGDIR_NAME));
TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logsDir,
snapshotServers, snapshotDescriptor, new Path(snapshotDir,
HConstants.HREGION_LOGDIR_NAME));
}
// check the table info
HTableDescriptor desc = FSTableDescriptors.getTableDescriptor(fs, rootDir, tableName);
HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
HTableDescriptor desc = FSTableDescriptors.getTableDescriptor(fs, rootDir,
tableName);
HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptor(fs,
snapshotDir);
assertEquals(desc, snapshotDesc);
// check the region snapshot for all the regions
@ -126,29 +196,36 @@ public class SnapshotTestingUtils {
for (HRegionInfo info : regions) {
String regionName = info.getEncodedName();
Path regionDir = new Path(snapshotDir, regionName);
HRegionInfo snapshotRegionInfo = HRegion.loadDotRegionInfoFileContent(fs, regionDir);
HRegionInfo snapshotRegionInfo = HRegion.loadDotRegionInfoFileContent(fs,
regionDir);
assertEquals(info, snapshotRegionInfo);
// check to make sure we have the family
Path familyDir = new Path(regionDir, Bytes.toString(testFamily));
assertTrue("Expected to find: " + familyDir + ", but it doesn't exist", fs.exists(familyDir));
// make sure we have some files references
assertTrue(fs.listStatus(familyDir).length > 0);
if (!familyEmpty) {
Path familyDir = new Path(regionDir, Bytes.toString(testFamily));
assertTrue("Expected to find: " + familyDir + ", but it doesn't exist",
fs.exists(familyDir));
// make sure we have some files references
assertTrue(fs.listStatus(familyDir).length > 0);
}
}
}
/**
* Helper method for testing async snapshot operations. Just waits for the given snapshot to
* complete on the server by repeatedly checking the master.
* @param master running the snapshot
* @param snapshot to check
* @param sleep amount to sleep between checks to see if the snapshot is done
* Helper method for testing async snapshot operations. Just waits for the
* given snapshot to complete on the server by repeatedly checking the master.
*
* @param master: the master running the snapshot
* @param snapshot: the snapshot to check
* @param sleep: amount to sleep between checks to see if the snapshot is done
* @throws ServiceException if the snapshot fails
*/
public static void waitForSnapshotToComplete(HMaster master, SnapshotDescription snapshot,
long sleep) throws ServiceException {
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
.build();
IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder().buildPartial();
public static void waitForSnapshotToComplete(HMaster master,
SnapshotDescription snapshot, long sleep) throws ServiceException {
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder()
.setSnapshot(snapshot).build();
IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder()
.buildPartial();
while (!done.getDone()) {
done = master.isSnapshotDone(null, request);
try {
@ -159,23 +236,28 @@ public class SnapshotTestingUtils {
}
}
public static void cleanupSnapshot(HBaseAdmin admin, byte[] tableName) throws IOException {
public static void cleanupSnapshot(HBaseAdmin admin, byte[] tableName)
throws IOException {
SnapshotTestingUtils.cleanupSnapshot(admin, Bytes.toString(tableName));
}
public static void cleanupSnapshot(HBaseAdmin admin, String snapshotName) throws IOException {
public static void cleanupSnapshot(HBaseAdmin admin, String snapshotName)
throws IOException {
// delete the taken snapshot
admin.deleteSnapshot(snapshotName);
assertNoSnapshots(admin);
}
/**
* Expect the snapshot to throw an error when checking if the snapshot is complete
* Expect the snapshot to throw an error when checking if the snapshot is
* complete
*
* @param master master to check
* @param snapshot the {@link SnapshotDescription} request to pass to the master
* @param clazz expected exception from the master
*/
public static void expectSnapshotDoneException(HMaster master, IsSnapshotDoneRequest snapshot,
public static void expectSnapshotDoneException(HMaster master,
IsSnapshotDoneRequest snapshot,
Class<? extends HBaseSnapshotException> clazz) {
try {
master.isSnapshotDone(null, snapshot);
@ -193,12 +275,14 @@ public class SnapshotTestingUtils {
/**
* List all the HFiles in the given table
* @param fs FileSystem where the table lives
*
* @param fs: FileSystem where the table lives
* @param tableDir directory of the table
* @return array of the current HFiles in the table (could be a zero-length array)
* @throws IOException on unexecpted error reading the FS
*/
public static FileStatus[] listHFiles(final FileSystem fs, Path tableDir) throws IOException {
public static FileStatus[] listHFiles(final FileSystem fs, Path tableDir)
throws IOException {
// setup the filters we will need based on the filesystem
PathFilter regionFilter = new FSUtils.RegionDirFilter(fs);
PathFilter familyFilter = new FSUtils.FamilyDirFilter(fs);
@ -215,16 +299,20 @@ public class SnapshotTestingUtils {
FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, regionFilter);
// if no regions, then we are done
if (regionDirs == null || regionDirs.length == 0) return new FileStatus[0];
if (regionDirs == null || regionDirs.length == 0)
return new FileStatus[0];
// go through each of the regions, and add al the hfiles under each family
List<FileStatus> regionFiles = new ArrayList<FileStatus>(regionDirs.length);
for (FileStatus regionDir : regionDirs) {
FileStatus[] fams = FSUtils.listStatus(fs, regionDir.getPath(), familyFilter);
FileStatus[] fams = FSUtils.listStatus(fs, regionDir.getPath(),
familyFilter);
// if no families, then we are done again
if (fams == null || fams.length == 0) continue;
if (fams == null || fams.length == 0)
continue;
// add all the hfiles under the family
regionFiles.addAll(SnapshotTestingUtils.getHFilesInRegion(fams, fs, fileFilter));
regionFiles.addAll(SnapshotTestingUtils.getHFilesInRegion(fams, fs,
fileFilter));
}
FileStatus[] files = new FileStatus[regionFiles.size()];
regionFiles.toArray(files);
@ -233,22 +321,117 @@ public class SnapshotTestingUtils {
/**
* Get all the hfiles in the region, under the passed set of families
* @param families all the family directories under the region
* @param fs filesystem where the families live
* @param fileFilter filter to only include files
*
* @param families: all the family directories under the region
* @param fs: filesystem where the families live
* @param fileFilter: filter to only include files
* @return collection of all the hfiles under all the passed in families (non-null)
* @throws IOException on unexecpted error reading the FS
*/
public static Collection<FileStatus> getHFilesInRegion(FileStatus[] families, FileSystem fs,
PathFilter fileFilter) throws IOException {
public static Collection<FileStatus> getHFilesInRegion(FileStatus[] families,
FileSystem fs, PathFilter fileFilter) throws IOException {
Set<FileStatus> files = new TreeSet<FileStatus>();
for (FileStatus family : families) {
// get all the hfiles in the family
FileStatus[] hfiles = FSUtils.listStatus(fs, family.getPath(), fileFilter);
FileStatus[] hfiles = FSUtils
.listStatus(fs, family.getPath(), fileFilter);
// if no hfiles, then we are done with this family
if (hfiles == null || hfiles.length == 0) continue;
if (hfiles == null || hfiles.length == 0)
continue;
files.addAll(Arrays.asList(hfiles));
}
return files;
}
/**
* Take an offline snapshot of the specified table and verify if the given
* family is empty. Note that this will leave the table disabled
* in the case of an offline snapshot.
*/
public static void createOfflineSnapshotAndValidate(HBaseAdmin admin,
String tableNameString, String familyName, String snapshotNameString,
Path rootDir, FileSystem fs, boolean familyEmpty) throws Exception {
createSnapshotAndValidate(admin, tableNameString, familyName,
snapshotNameString, rootDir, fs, familyEmpty, false);
}
/**
* Take a snapshot of the specified table and verify if the given family is
* empty. Note that this will leave the table disabled
* in the case of an offline snapshot.
*/
public static void createSnapshotAndValidate(HBaseAdmin admin,
String tableNameString, String familyName, String snapshotNameString,
Path rootDir, FileSystem fs, boolean familyEmpty, boolean onlineSnapshot)
throws Exception {
byte[] tableName = Bytes.toBytes(tableNameString);
if (!onlineSnapshot) {
try {
admin.disableTable(tableNameString);
} catch (TableNotEnabledException tne) {
LOG.info("In attempting to disable " + tableNameString
+ " it turns out that this table is already disabled.");
}
}
admin.snapshot(snapshotNameString, tableNameString);
List<SnapshotDescription> snapshots = SnapshotTestingUtils
.assertExistsMatchingSnapshot(admin, snapshotNameString,
tableNameString);
if (snapshots == null || snapshots.size() != 1) {
Assert.fail("Incorrect number of snapshots for table "
+ String.valueOf(tableNameString));
}
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName,
Bytes.toBytes(familyName), rootDir, admin, fs, false, new Path(rootDir,
HConstants.HREGION_LOGDIR_NAME), familyEmpty, null);
}
public static void createSnapshotAndValidate(HBaseAdmin admin,
String tableNameString, String familyName, String snapshotNameString,
Path rootDir, FileSystem fs) throws Exception {
createSnapshotAndValidate(admin, tableNameString, familyName,
snapshotNameString, rootDir, fs, false, false);
}
/**
* This will create a snapshot. Note that this has the side effect
* of leaving the input table disabled if the offline snapshot
* option is chosen.
*
*/
public static void createSnapshotAndValidate(HBaseAdmin admin,
String tableNameString, String familyName, String snapshotNameString,
Path rootDir, FileSystem fs, boolean online) throws Exception {
createSnapshotAndValidate(admin, tableNameString, familyName,
snapshotNameString, rootDir, fs, false, online);
}
public static void createSnapshotAndValidate(HBaseAdmin admin,
String tableNameString, List<byte[]> nonEmptyFamilyNames, List<byte[]> emptyFamilyNames,
String snapshotNameString, Path rootDir, FileSystem fs) throws Exception {
byte[] tableName = Bytes.toBytes(tableNameString);
try {
admin.disableTable(tableNameString);
} catch (TableNotEnabledException tne) {
LOG.info("In attempting to disable " + tableNameString + " it turns out that the this table is already disabled.");
}
admin.snapshot(snapshotNameString, tableNameString);
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertExistsMatchingSnapshot(admin,
snapshotNameString, tableNameString);
// Create test-timestamp-clone
if (snapshots == null || snapshots.size() != 1) {
Assert.fail("Incorrect number of snapshots for table " + String.valueOf(tableNameString));
}
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName, nonEmptyFamilyNames, emptyFamilyNames,
rootDir, admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), null);
}
}