You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 19:05:55 UTC

svn commit: r1445782 [3/3] - in /hbase/branches/hbase-7290: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ hbase-protocol/src/main/protobuf/ hbase-server/src/main/java/org/ap...

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,160 @@
+/**
+ * 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.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.google.protobuf.RpcController;
+
+/**
+ * Test snapshot logic from the client
+ */
+@Category(SmallTests.class)
+public class TestSnapshotFromAdmin {
+
+  private static final Log LOG = LogFactory.getLog(TestSnapshotFromAdmin.class);
+
+  /**
+   * Test that the logic for doing 'correct' back-off based on exponential increase and the max-time
+   * passed from the server ensures the correct overall waiting for the snapshot to finish.
+   * @throws Exception
+   */
+  @Test(timeout = 10000)
+  public void testBackoffLogic() throws Exception {
+    final int maxWaitTime = 7500;
+    final int numRetries = 10;
+    final int pauseTime = 500;
+    // calculate the wait time, if we just do straight backoff (ignoring the expected time from
+    // master)
+    long ignoreExpectedTime = 0;
+    for (int i = 0; i < 6; i++) {
+      ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime;
+    }
+    // the correct wait time, capping at the maxTime/tries + fudge room
+    final long time = pauseTime * 3 + ((maxWaitTime / numRetries) * 3) + 300;
+    assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time "
+        + "- further testing won't prove anything.", time < ignoreExpectedTime);
+
+    // setup the mocks
+    HConnectionManager.HConnectionImplementation mockConnection = Mockito
+        .mock(HConnectionManager.HConnectionImplementation.class);
+    Configuration conf = HBaseConfiguration.create();
+    // setup the conf to match the expected properties
+    conf.setInt("hbase.client.retries.number", numRetries);
+    conf.setLong("hbase.client.pause", pauseTime);
+    // mock the master admin to our mock
+    MasterAdminKeepAliveConnection mockMaster = Mockito.mock(MasterAdminKeepAliveConnection.class);
+    Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
+    Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(mockMaster);
+    // set the max wait time for the snapshot to complete
+    TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder()
+        .setExpectedTimeout(maxWaitTime)
+        .build();
+    Mockito
+        .when(
+          mockMaster.snapshot((RpcController) Mockito.isNull(),
+            Mockito.any(TakeSnapshotRequest.class))).thenReturn(response);
+    // setup the response
+    IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
+    builder.setDone(false);
+    // first five times, we return false, last we get success
+    Mockito.when(
+      mockMaster.isSnapshotDone((RpcController) Mockito.isNull(),
+        Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(builder.build(), builder.build(),
+      builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
+
+    // setup the admin and run the test
+    HBaseAdmin admin = new HBaseAdmin(mockConnection);
+    String snapshot = "snapshot";
+    String table = "table";
+    // get start time
+    long start = System.currentTimeMillis();
+    admin.snapshot(snapshot, table);
+    long finish = System.currentTimeMillis();
+    long elapsed = (finish - start);
+    assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time);
+    admin.close();
+  }
+
+  /**
+   * Make sure that we validate the snapshot name and the table name before we pass anything across
+   * the wire
+   * @throws Exception on failure
+   */
+  @Test
+  public void testValidateSnapshotName() throws Exception {
+    HConnectionManager.HConnectionImplementation mockConnection = Mockito
+        .mock(HConnectionManager.HConnectionImplementation.class);
+    Configuration conf = HBaseConfiguration.create();
+    Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
+    HBaseAdmin admin = new HBaseAdmin(mockConnection);
+    SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
+    // check that invalid snapshot names fail
+    failSnapshotStart(admin, builder.setName(".snapshot").build());
+    failSnapshotStart(admin, builder.setName("-snapshot").build());
+    failSnapshotStart(admin, builder.setName("snapshot fails").build());
+    failSnapshotStart(admin, builder.setName("snap$hot").build());
+    // check the table name also get verified
+    failSnapshotStart(admin, builder.setName("snapshot").setTable(".table").build());
+    failSnapshotStart(admin, builder.setName("snapshot").setTable("-table").build());
+    failSnapshotStart(admin, builder.setName("snapshot").setTable("table fails").build());
+    failSnapshotStart(admin, builder.setName("snapshot").setTable("tab%le").build());
+
+    // mock the master connection
+    MasterAdminKeepAliveConnection master = Mockito.mock(MasterAdminKeepAliveConnection.class);
+    Mockito.when(mockConnection.getKeepAliveMasterAdmin()).thenReturn(master);
+    TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder().setExpectedTimeout(0).build();
+    Mockito.when(
+      master.snapshot((RpcController) Mockito.isNull(), Mockito.any(TakeSnapshotRequest.class)))
+        .thenReturn(response);
+    IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
+    Mockito.when(
+      master.isSnapshotDone((RpcController) Mockito.isNull(),
+        Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
+
+      // make sure that we can use valid names
+    admin.snapshot(builder.setName("snapshot").setTable("table").build());
+  }
+
+  private void failSnapshotStart(HBaseAdmin admin, SnapshotDescription snapshot) throws IOException {
+    try {
+      admin.snapshot(snapshot);
+      fail("Snapshot should not have succeed with name:" + snapshot.getName());
+    } catch (IllegalArgumentException e) {
+      LOG.debug("Correctly failed to start snapshot:" + e.getMessage());
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,199 @@
+/**
+ * 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.fail;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+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.HConstants;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test create/using/deleting snapshots from the client
+ * <p>
+ * This is an end-to-end test for the snapshot utility
+ */
+@Category(LargeTests.class)
+public class TestSnapshotFromClient {
+  private static final Log LOG = LogFactory.getLog(TestSnapshotFromClient.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 byte[] TEST_FAM = Bytes.toBytes("fam");
+  private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
+
+  /**
+   * Setup the config for the cluster
+   * @throws Exception on failure
+   */
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(NUM_RS);
+  }
+
+  private static void setupConf(Configuration conf) {
+    // 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.client.retries.number", 1);
+  }
+
+  @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);
+    }
+  }
+
+  /**
+   * Test snapshotting a table that is offline
+   * @throws Exception
+   */
+  @Test
+  public void testOfflineTableSnapshot() throws Exception {
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    // make sure we don't fail on listing snapshots
+    SnapshotTestingUtils.assertNoSnapshots(admin);
+
+    // put some stuff in the table
+    HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
+    UTIL.loadTable(table, TEST_FAM);
+
+    // get the name of all the regionservers hosting the snapshotted table
+    Set<String> snapshotServers = new HashSet<String>();
+    List<RegionServerThread> servers = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
+    for (RegionServerThread server : servers) {
+      if (server.getRegionServer().getOnlineRegions(TABLE_NAME).size() > 0) {
+        snapshotServers.add(server.getRegionServer().getServerName().toString());
+      }
+    }
+
+    LOG.debug("FS state before disable:");
+    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    // XXX if this is flakey, might want to consider using the async version and looping as
+    // disableTable can succeed and still timeout.
+    admin.disableTable(TABLE_NAME);
+
+    LOG.debug("FS state before snapshot:");
+    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+
+    // take a snapshot of the disabled table
+    byte[] snapshot = Bytes.toBytes("offlineTableSnapshot");
+    admin.snapshot(snapshot, TABLE_NAME);
+    LOG.debug("Snapshot completed.");
+
+    // make sure we have the snapshot
+    List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
+      snapshot, TABLE_NAME);
+
+    // make sure its a valid snapshot
+    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+    LOG.debug("FS state after snapshot:");
+    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+
+    SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir,
+      admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), snapshotServers);
+
+    admin.deleteSnapshot(snapshot);
+    snapshots = admin.listSnapshots();
+    SnapshotTestingUtils.assertNoSnapshots(admin);
+  }
+
+  @Test
+  public void testSnapshotFailsOnNonExistantTable() throws Exception {
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    // make sure we don't fail on listing snapshots
+    SnapshotTestingUtils.assertNoSnapshots(admin);
+    String tableName = "_not_a_table";
+
+    // make sure the table doesn't exist
+    boolean fail = false;
+    do {
+    try {
+    admin.getTableDescriptor(Bytes.toBytes(tableName));
+    fail = true;
+        LOG.error("Table:" + tableName + " already exists, checking a new name");
+    tableName = tableName+"!";
+    }catch(TableNotFoundException e) {
+      fail = false;
+      }
+    } while (fail);
+
+    // snapshot the non-existant table
+    try {
+      admin.snapshot("fail", tableName);
+      fail("Snapshot succeeded even though there is not table.");
+    } catch (SnapshotCreationException e) {
+      LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSnapshotFromMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSnapshotFromMaster.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSnapshotFromMaster.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSnapshotFromMaster.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.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.MediumTests;
+import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ListSnapshotRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ListSnapshotResponse;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
+import org.apache.hadoop.hbase.snapshot.exception.UnknownSnapshotException;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Test the master-related aspects of a snapshot
+ */
+@Category(MediumTests.class)
+public class TestSnapshotFromMaster {
+
+  private static final Log LOG = LogFactory.getLog(TestSnapshotFromMaster.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static final int NUM_RS = 2;
+  private static Path rootDir;
+  private static Path snapshots;
+  private static FileSystem fs;
+  private static HMaster master;
+
+  /**
+   * Setup the config for the cluster
+   */
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(NUM_RS);
+    fs = UTIL.getDFSCluster().getFileSystem();
+    rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
+    snapshots = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
+    master = UTIL.getMiniHBaseCluster().getMaster();
+  }
+
+  private static void setupConf(Configuration conf) {
+    // 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", 5);
+    conf.setInt("hbase.hstore.compactionThreshold", 5);
+    // 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.client.retries.number", 1);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(null);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (!fs.delete(snapshots, true)) {
+      throw new IOException("Couldn't delete snapshots directory (" + snapshots
+          + " for an unknown reason");
+    }
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      // NOOP;
+    }
+  }
+
+  /**
+   * Test that the contract from the master for checking on a snapshot are valid.
+   * <p>
+   * <ol>
+   * <li>If a snapshot fails with an error, we expect to get the source error.</li>
+   * <li>If there is no snapshot name supplied, we should get an error.</li>
+   * <li>If asking about a snapshot has hasn't occurred, you should get an error.</li>
+   * </ol>
+   */
+  @Test(timeout = 15000)
+  public void testIsDoneContract() throws Exception {
+
+    IsSnapshotDoneRequest.Builder builder = IsSnapshotDoneRequest.newBuilder();
+    String snapshotName = "asyncExpectedFailureTest";
+
+    // check that we get an exception when looking up snapshot where one hasn't happened
+    SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
+      UnknownSnapshotException.class);
+
+    // and that we get the same issue, even if we specify a name
+    SnapshotDescription desc = SnapshotDescription.newBuilder().setName(snapshotName).build();
+    builder.setSnapshot(desc);
+    SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
+      UnknownSnapshotException.class);
+
+    // set a mock handler to simulate a snapshot
+    DisabledTableSnapshotHandler mockHandler = Mockito.mock(DisabledTableSnapshotHandler.class);
+    Mockito.when(mockHandler.getExceptionIfFailed()).thenReturn(null);
+    Mockito.when(mockHandler.getSnapshot()).thenReturn(desc);
+    Mockito.when(mockHandler.isFinished()).thenReturn(new Boolean(true));
+
+    master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(mockHandler);
+
+    // if we do a lookup without a snapshot name, we should fail - you should always know your name
+    builder = IsSnapshotDoneRequest.newBuilder();
+    SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
+      UnknownSnapshotException.class);
+
+    // then do the lookup for the snapshot that it is done
+    builder.setSnapshot(desc);
+    IsSnapshotDoneResponse response = master.isSnapshotDone(null, builder.build());
+    assertTrue("Snapshot didn't complete when it should have.", response.getDone());
+
+    // now try the case where we are looking for a snapshot we didn't take
+    builder.setSnapshot(SnapshotDescription.newBuilder().setName("Not A Snapshot").build());
+    SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
+      UnknownSnapshotException.class);
+
+    // then create a snapshot to the fs and make sure that we can find it when checking done
+    snapshotName = "completed";
+    FileSystem fs = master.getMasterFileSystem().getFileSystem();
+    Path root = master.getMasterFileSystem().getRootDir();
+    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, root);
+    desc = desc.toBuilder().setName(snapshotName).build();
+    SnapshotDescriptionUtils.writeSnapshotInfo(desc, snapshotDir, fs);
+
+    builder.setSnapshot(desc);
+    response = master.isSnapshotDone(null, builder.build());
+    assertTrue("Completed, on-disk snapshot not found", response.getDone());
+    
+    HBaseSnapshotException testException = new SnapshotCreationException("test fail", desc);
+    Mockito.when(mockHandler.getExceptionIfFailed()).thenReturn(testException);
+    try {
+      master.isSnapshotDone(null, builder.build());
+      fail("Master should have passed along snapshot error, but didn't");
+    }catch(ServiceException e) {
+      LOG.debug("Correctly got exception back from the master on failure: " + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testListSnapshots() throws Exception {
+    // first check when there are no snapshots
+    ListSnapshotRequest request = ListSnapshotRequest.newBuilder().build();
+    ListSnapshotResponse response = master.listSnapshots(null, request);
+    assertEquals("Found unexpected number of snapshots", 0, response.getSnapshotsCount());
+
+    // write one snapshot to the fs
+    String snapshotName = "completed";
+    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
+    SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
+
+    // check that we get one snapshot
+    response = master.listSnapshots(null, request);
+    assertEquals("Found unexpected number of snapshots", 1, response.getSnapshotsCount());
+    List<SnapshotDescription> snapshots = response.getSnapshotsList();
+    List<SnapshotDescription> expected = Lists.newArrayList(snapshot);
+    assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
+
+    // write a second snapshot
+    snapshotName = "completed_two";
+    snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
+    snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
+    SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
+    expected.add(snapshot);
+
+    // check that we get one snapshot
+    response = master.listSnapshots(null, request);
+    assertEquals("Found unexpected number of snapshots", 2, response.getSnapshotsCount());
+    snapshots = response.getSnapshotsList();
+    assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
+  }
+
+  @Test
+  public void testDeleteSnapshot() throws Exception {
+
+    String snapshotName = "completed";
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
+
+    DeleteSnapshotRequest request = DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
+        .build();
+    try {
+      master.deleteSnapshot(null, request);
+      fail("Master didn't throw exception when attempting to delete snapshot that doesn't exist");
+    } catch (ServiceException e) {
+      LOG.debug("Correctly failed delete of non-existant snapshot:" + e.getMessage());
+    }
+
+    // write one snapshot to the fs
+    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
+    SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
+
+    // then delete the existing snapshot,which shouldn't cause an exception to be thrown
+    master.deleteSnapshot(null, request);
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/manage/TestSnapshotManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/manage/TestSnapshotManager.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/manage/TestSnapshotManager.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/manage/TestSnapshotManager.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.snapshot.manage;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.SnapshotSentinel;
+import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test basic snapshot manager functionality
+ */
+@Category(SmallTests.class)
+public class TestSnapshotManager {
+  private static final Log LOG = LogFactory.getLog(TestSnapshotManager.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  MasterServices services = Mockito.mock(MasterServices.class);
+  ZooKeeperWatcher watcher = Mockito.mock(ZooKeeperWatcher.class);
+  ExecutorService pool = Mockito.mock(ExecutorService.class);
+  MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
+  FileSystem fs;
+  {
+    try {
+      fs = UTIL.getTestFileSystem();
+    } catch (IOException e) {
+      throw new RuntimeException("Couldn't get test filesystem", e);
+    }
+
+  }
+
+  private SnapshotManager getNewManager() throws KeeperException {
+    Mockito.reset(services, watcher, pool);
+    Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
+    Mockito.when(mfs.getFileSystem()).thenReturn(fs);
+    Mockito.when(mfs.getRootDir()).thenReturn(UTIL.getDataTestDir());
+    return new SnapshotManager(services, watcher, pool);
+  }
+
+
+
+  @Test
+  public void testInProcess() throws KeeperException, SnapshotCreationException {
+    SnapshotManager manager = getNewManager();
+    SnapshotSentinel handler = Mockito.mock(SnapshotSentinel.class);
+    assertFalse("Manager is in process when there is no current handler", manager.isTakingSnapshot());
+    manager.setSnapshotHandlerForTesting(handler);
+    Mockito.when(handler.isFinished()).thenReturn(false);
+    assertTrue("Manager isn't in process when handler is running", manager.isTakingSnapshot());
+    Mockito.when(handler.isFinished()).thenReturn(true);
+    assertFalse("Manager is process when handler isn't running", manager.isTakingSnapshot());
+  }
+
+  /**
+   * Test that we stop the running disabled table snapshot by passing along an error to the error
+   * handler.
+   * @throws Exception
+   */
+  @Test
+  public void testStopPropagation() throws Exception {
+    // create a new orchestrator and hook up a listener
+    SnapshotManager manager = getNewManager();
+    FSUtils.setRootDir(UTIL.getConfiguration(), UTIL.getDataTestDir());
+
+    // setup a mock snapshot to run
+    String tableName = "some table";
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("testAbort")
+        .setTable(tableName).build();
+    // mock out all the expected call to the master services
+    // this allows us to run must faster and without using a minicluster
+
+    // ensure the table exists when we ask for it
+    TableDescriptors tables = Mockito.mock(TableDescriptors.class);
+    Mockito.when(services.getTableDescriptors()).thenReturn(tables);
+    HTableDescriptor descriptor = Mockito.mock(HTableDescriptor.class);
+    Mockito.when(tables.get(Mockito.anyString())).thenReturn(descriptor);
+
+    // return the local file system as the backing to the MasterFileSystem
+    MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
+    Mockito.when(mfs.getFileSystem()).thenReturn(UTIL.getTestFileSystem());
+    Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
+    Mockito.when(services.getConfiguration()).thenReturn(UTIL.getConfiguration());
+
+    // create a new handler that we will check for errors
+    manager.snapshotDisabledTable(snapshot);
+    // make sure we submitted the handler, but because its mocked, it doesn't run it.
+    Mockito.verify(pool, Mockito.times(1)).submit(Mockito.any(DisabledTableSnapshotHandler.class));
+
+    // pass along the stop notification
+    manager.stop("stopping for test");
+    SnapshotSentinel handler = manager.getCurrentSnapshotSentinel();
+    assertNotNull("Snare didn't receive error notification from snapshot manager.",
+      handler.getExceptionIfFailed());
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/error/TestSnapshotExceptionSnare.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/error/TestSnapshotExceptionSnare.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/error/TestSnapshotExceptionSnare.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/error/TestSnapshotExceptionSnare.java Wed Feb 13 18:05:53 2013
@@ -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.server.snapshot.error;
+
+import static org.junit.Assert.fail;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.errorhandling.ExceptionListener;
+import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
+import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+import org.junit.Test;
+
+/**
+ * Test the exception snare propagates errors as expected
+ */
+public class TestSnapshotExceptionSnare {
+
+  private static final Log LOG = LogFactory.getLog(TestSnapshotExceptionSnare.class);
+
+  /**
+   * This test ensures that we only propagate snapshot exceptions, even if we don't get a snapshot
+   * exception
+   */
+  @Test
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public void testPropagatesOnlySnapshotException() {
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
+    ExceptionListener snare = new SnapshotExceptionSnare(snapshot);
+    snare.receiveError("Some message", new Exception());
+    try {
+      ((SnapshotExceptionSnare) snare).failOnError();
+      fail("Snare didn't throw an exception");
+    } catch (HBaseSnapshotException e) {
+      LOG.error("Correctly got a snapshot exception" + e);
+    }
+  }
+
+  @Test
+  public void testPropatesTimerError() {
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
+    SnapshotExceptionSnare snare = new SnapshotExceptionSnare(snapshot);
+    Configuration conf = new Configuration();
+    // don't let the timer count down before we fire it off
+    conf.setLong(SnapshotDescriptionUtils.MASTER_WAIT_TIME_DISABLED_SNAPSHOT, Long.MAX_VALUE);
+    OperationAttemptTimer timer = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf,
+      snare);
+    timer.trigger();
+    try {
+      snare.failOnError();
+    } catch (HBaseSnapshotException e) {
+      LOG.info("Correctly failed from timer:" + e.getMessage());
+    }
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestCopyRecoveredEditsTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestCopyRecoveredEditsTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestCopyRecoveredEditsTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestCopyRecoveredEditsTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,128 @@
+/**
+ * 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.server.snapshot.task;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test that we correctly copy the recovered edits from a directory
+ */
+@Category(SmallTests.class)
+public class TestCopyRecoveredEditsTask {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Test
+  public void testCopyFiles() throws Exception {
+
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
+    SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
+    FileSystem fs = UTIL.getTestFileSystem();
+    Path root = UTIL.getDataTestDir();
+    String regionName = "regionA";
+    Path regionDir = new Path(root, regionName);
+    Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, root);
+
+    try {
+      // doesn't really matter where the region's snapshot directory is, but this is pretty close
+      Path snapshotRegionDir = new Path(workingDir, regionName);
+      fs.mkdirs(snapshotRegionDir);
+
+      // put some stuff in the recovered.edits directory
+      Path edits = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
+      fs.mkdirs(edits);
+      // make a file with some data
+      Path file1 = new Path(edits, "0000000000000002352");
+      FSDataOutputStream out = fs.create(file1);
+      byte[] data = new byte[] { 1, 2, 3, 4 };
+      out.write(data);
+      out.close();
+      // make an empty file
+      Path empty = new Path(edits, "empty");
+      fs.createNewFile(empty);
+
+      CopyRecoveredEditsTask task = new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir,
+          snapshotRegionDir);
+      task.run();
+
+      Path snapshotEdits = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
+      FileStatus[] snapshotEditFiles = FSUtils.listStatus(fs, snapshotEdits);
+      assertEquals("Got wrong number of files in the snapshot edits", 1, snapshotEditFiles.length);
+      FileStatus file = snapshotEditFiles[0];
+      assertEquals("Didn't copy expected file", file1.getName(), file.getPath().getName());
+
+      Mockito.verify(monitor, Mockito.never()).receiveError(Mockito.anyString(),
+        Mockito.any(HBaseSnapshotException.class));
+      Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
+        Mockito.any(SnapshotDescription.class));
+      Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
+        Mockito.any(SnapshotDescription.class), Mockito.any(Exception.class));
+    } finally {
+      // cleanup the working directory
+      FSUtils.delete(fs, regionDir, true);
+      FSUtils.delete(fs, workingDir, true);
+    }
+  }
+
+  /**
+   * Check that we don't get an exception if there is no recovered edits directory to copy
+   * @throws Exception on failure
+   */
+  @Test
+  public void testNoEditsDir() throws Exception {
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
+    SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
+    FileSystem fs = UTIL.getTestFileSystem();
+    Path root = UTIL.getDataTestDir();
+    String regionName = "regionA";
+    Path regionDir = new Path(root, regionName);
+    Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, root);
+    try {
+      // doesn't really matter where the region's snapshot directory is, but this is pretty close
+      Path snapshotRegionDir = new Path(workingDir, regionName);
+      fs.mkdirs(snapshotRegionDir);
+      Path regionEdits = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
+      assertFalse("Edits dir exists already - it shouldn't", fs.exists(regionEdits));
+
+      CopyRecoveredEditsTask task = new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir,
+          snapshotRegionDir);
+      task.run();
+    } finally {
+      // cleanup the working directory
+      FSUtils.delete(fs, regionDir, true);
+      FSUtils.delete(fs, workingDir, true);
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestReferenceRegionHFilesTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestReferenceRegionHFilesTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestReferenceRegionHFilesTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestReferenceRegionHFilesTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,92 @@
+/**
+ * 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.server.snapshot.task;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category(SmallTests.class)
+public class TestReferenceRegionHFilesTask {
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Test
+  public void testRun() throws IOException {
+    FileSystem fs = UTIL.getTestFileSystem();
+    // setup the region internals
+    Path testdir = UTIL.getDataTestDir();
+    Path regionDir = new Path(testdir, "region");
+    Path family1 = new Path(regionDir, "fam1");
+    // make an empty family
+    Path family2 = new Path(regionDir, "fam2");
+    fs.mkdirs(family2);
+
+    // add some files to family 1
+    Path file1 = new Path(family1, "05f99689ae254693836613d1884c6b63");
+    fs.createNewFile(file1);
+    Path file2 = new Path(family1, "7ac9898bf41d445aa0003e3d699d5d26");
+    fs.createNewFile(file2);
+
+    // create the snapshot directory
+    Path snapshotRegionDir = new Path(testdir, HConstants.SNAPSHOT_DIR_NAME);
+    fs.mkdirs(snapshotRegionDir);
+
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("name")
+        .setTable("table").build();
+    SnapshotExceptionSnare monitor = Mockito.mock(SnapshotExceptionSnare.class);
+    ReferenceRegionHFilesTask task = new ReferenceRegionHFilesTask(snapshot, monitor, regionDir,
+        fs, snapshotRegionDir);
+    task.run();
+
+    // make sure we never get an error
+    Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
+      Mockito.eq(snapshot));
+    Mockito.verify(monitor, Mockito.never()).snapshotFailure(Mockito.anyString(),
+      Mockito.eq(snapshot), Mockito.any(Exception.class));
+
+    // verify that all the hfiles get referenced
+    List<String> hfiles = new ArrayList<String>(2);
+    FileStatus[] regions = FSUtils.listStatus(fs, snapshotRegionDir);
+    for (FileStatus region : regions) {
+      FileStatus[] fams = FSUtils.listStatus(fs, region.getPath());
+      for (FileStatus fam : fams) {
+        FileStatus[] files = FSUtils.listStatus(fs, fam.getPath());
+        for (FileStatus file : files) {
+          hfiles.add(file.getPath().getName());
+        }
+      }
+    }
+    assertTrue("Didn't reference :" + file1, hfiles.contains(file1.getName()));
+    assertTrue("Didn't reference :" + file1, hfiles.contains(file2.getName()));
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestSnapshotTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestSnapshotTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestSnapshotTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestSnapshotTask.java Wed Feb 13 18:05:53 2013
@@ -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.server.snapshot.task;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category(SmallTests.class)
+public class TestSnapshotTask {
+
+  /**
+   * Check that errors from running the task get propagated back to the error listener.
+   */
+  @Test
+  public void testErrorPropagationg() {
+    SnapshotExceptionSnare error = Mockito.mock(SnapshotExceptionSnare.class);
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot")
+        .setTable("table").build();
+    final Exception thrown = new Exception("Failed!");
+    SnapshotTask fail = new SnapshotTask(snapshot, error, "always fails") {
+
+      @Override
+      protected void process() throws Exception {
+        throw thrown;
+      }
+    };
+    fail.run();
+
+    Mockito.verify(error, Mockito.times(1)).snapshotFailure(Mockito.anyString(),
+      Mockito.eq(snapshot), Mockito.eq(thrown));
+  }
+
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestWALReferenceTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestWALReferenceTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestWALReferenceTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/snapshot/task/TestWALReferenceTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,99 @@
+/**
+ * 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.server.snapshot.task;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+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.HConstants;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test that the WAL reference task works as expected
+ */
+public class TestWALReferenceTask {
+
+  private static final Log LOG = LogFactory.getLog(TestWALReferenceTask.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Test
+  public void testRun() throws IOException {
+    Configuration conf = UTIL.getConfiguration();
+    FileSystem fs = UTIL.getTestFileSystem();
+    // setup the log dir
+    Path testDir = UTIL.getDataTestDir();
+    Set<String> servers = new HashSet<String>();
+    Path logDir = new Path(testDir, ".logs");
+    Path server1Dir = new Path(logDir, "Server1");
+    servers.add(server1Dir.getName());
+    Path server2Dir = new Path(logDir, "me.hbase.com,56073,1348618509968");
+    servers.add(server2Dir.getName());
+    // logs under server 1
+    Path log1_1 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1348618520536");
+    Path log1_2 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
+    // logs under server 2
+    Path log2_1 = new Path(server2Dir, "me.hbase.com%2C56074%2C1348618509998.1348618515589");
+    Path log2_2 = new Path(server2Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
+
+    // create all the log files
+    fs.createNewFile(log1_1);
+    fs.createNewFile(log1_2);
+    fs.createNewFile(log2_1);
+    fs.createNewFile(log2_2);
+
+    FSUtils.logFileSystemState(fs, testDir, LOG);
+    FSUtils.setRootDir(conf, testDir);
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+        .setName("testWALReferenceSnapshot").build();
+    SnapshotExceptionSnare listener = Mockito.mock(SnapshotExceptionSnare.class);
+
+    // reference all the files in the first server directory
+    ReferenceServerWALsTask task = new ReferenceServerWALsTask(snapshot, listener, server1Dir,
+        conf, fs);
+    task.run();
+
+    // reference all the files in the first server directory
+    task = new ReferenceServerWALsTask(snapshot, listener, server2Dir, conf, fs);
+    task.run();
+
+    // verify that we got everything
+    FSUtils.logFileSystemState(fs, testDir, LOG);
+    Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, testDir);
+    Path snapshotLogDir = new Path(workingDir, HConstants.HREGION_LOGDIR_NAME);
+
+    // make sure we reference the all the wal files
+    TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logDir, servers, snapshot, snapshotLogDir);
+
+    // make sure we never got an error
+    Mockito.verify(listener, Mockito.atLeastOnce()).failOnError();
+    Mockito.verifyNoMoreInteractions(listener);
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,188 @@
+/**
+ * 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.snapshot;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.junit.Assert;
+
+import com.google.protobuf.ServiceException;
+
+/**
+ * Utilities class for snapshots
+ */
+public class SnapshotTestingUtils {
+
+  private static final Log LOG = LogFactory.getLog(SnapshotTestingUtils.class);
+
+  /**
+   * Assert that we don't have any snapshots lists
+   * @throws IOException if the admin operation fails
+   */
+  public static void assertNoSnapshots(HBaseAdmin admin) throws IOException {
+    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.
+   */
+  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.
+   */
+  public static List<SnapshotDescription> assertOneSnapshotThatMatches(HBaseAdmin admin,
+      String snapshotName, String tableName) throws IOException {
+    // list the snapshot
+    List<SnapshotDescription> snapshots = admin.listSnapshots();
+
+    assertEquals("Should only have 1 snapshot", 1, snapshots.size());
+    assertEquals(snapshotName, snapshots.get(0).getName());
+    assertEquals(tableName, snapshots.get(0).getTable());
+
+    return snapshots;
+  }
+
+  /**
+   * 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));
+  }
+
+  /**
+   * 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 {
+    Path snapshotDir = SnapshotDescriptionUtils
+        .getCompletedSnapshotDir(snapshotDescriptor, rootDir);
+    assertTrue(fs.exists(snapshotDir));
+    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));
+    }
+    // check the table info
+    HTableDescriptor desc = FSTableDescriptors.getTableDescriptor(fs, rootDir, tableName);
+    HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptor(fs, snapshotDir);
+    assertEquals(desc, snapshotDesc);
+
+    // check the region snapshot for all the regions
+    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    for (HRegionInfo info : regions) {
+      String regionName = info.getEncodedName();
+      Path regionDir = new Path(snapshotDir, regionName);
+      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);
+    }
+  }
+
+  /**
+   * 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
+   * @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();
+    while (!done.getDone()) {
+      done = master.isSnapshotDone(null, request);
+      try {
+        Thread.sleep(sleep);
+      } catch (InterruptedException e) {
+        throw new ServiceException(e);
+      }
+    }
+  }
+
+  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 {
+    // delete the taken snapshot
+    admin.deleteSnapshot(snapshotName);
+    assertNoSnapshots(admin);
+  }
+
+  /**
+   * 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,
+      Class<? extends HBaseSnapshotException> clazz) {
+    try {
+      master.isSnapshotDone(null, snapshot);
+      Assert.fail("didn't fail to lookup a snapshot");
+    } catch (ServiceException se) {
+      try {
+        throw ProtobufUtil.getRemoteException(se);
+      } catch (HBaseSnapshotException e) {
+        assertEquals("Threw wrong snapshot exception!", clazz, e.getClass());
+      } catch (Throwable t) {
+        Assert.fail("Threw an unexpected exception:" + t);
+      }
+    }
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,138 @@
+/**
+ * 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.snapshot;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+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.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
+import org.apache.hadoop.hbase.util.EnvironmentEdge;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test that the {@link SnapshotDescription} helper is helping correctly.
+ */
+@Category(SmallTests.class)
+public class TestSnapshotDescriptionUtils {
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static FileSystem fs;
+  private static Path root;
+
+  @BeforeClass
+  public static void setupFS() throws Exception {
+    fs = UTIL.getTestFileSystem();
+    root = new Path(UTIL.getDataTestDir(), "hbase");
+  }
+
+  @After
+  public void cleanupFS() throws Exception {
+    if (fs.exists(root)) {
+    if (!fs.delete(root, true)) {
+      throw new IOException("Failed to delete root test dir: " + root);
+    }
+    if (!fs.mkdirs(root)) {
+      throw new IOException("Failed to create root test dir: " + root);
+    }
+    }
+  }
+
+  private static final Log LOG = LogFactory.getLog(TestSnapshotDescriptionUtils.class);
+
+  @Test
+  public void testValidateDescriptor() {
+    EnvironmentEdge edge = new EnvironmentEdge() {
+      @Override
+      public long currentTimeMillis() {
+        return 0;
+      }
+    };
+    EnvironmentEdgeManagerTestHelper.injectEdge(edge);
+
+    // check a basic snapshot description
+    SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
+    builder.setName("snapshot");
+    builder.setTable("table");
+
+    // check that time is to an amount in the future
+    Configuration conf = new Configuration(false);
+    conf.setLong(SnapshotDescriptionUtils.TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION, 1);
+    SnapshotDescription desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
+    assertEquals("Description creation time wasn't set correctly", 1, desc.getCreationTime());
+
+    // test a global snapshot
+    edge = new EnvironmentEdge() {
+      @Override
+      public long currentTimeMillis() {
+        return 2;
+      }
+    };
+    EnvironmentEdgeManagerTestHelper.injectEdge(edge);
+    builder.setType(Type.GLOBAL);
+    desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
+    assertEquals("Description creation time wasn't set correctly", 2, desc.getCreationTime());
+
+    // test that we don't override a given value
+    builder.setCreationTime(10);
+    desc = SnapshotDescriptionUtils.validate(builder.build(), conf);
+    assertEquals("Description creation time wasn't set correctly", 10, desc.getCreationTime());
+
+    try {
+      SnapshotDescriptionUtils.validate(SnapshotDescription.newBuilder().setName("fail").build(),
+        conf);
+      fail("Snapshot was considered valid without a table name");
+    } catch (IllegalArgumentException e) {
+      LOG.debug("Correctly failed when snapshot doesn't have a tablename");
+    }
+  }
+
+  /**
+   * Test that we throw an exception if there is no working snapshot directory when we attempt to
+   * 'complete' the snapshot
+   * @throws Exception on failure
+   */
+  @Test
+  public void testCompleteSnapshotWithNoSnapshotDirectoryFailure() throws Exception {
+    Path snapshotDir = new Path(root, ".snapshot");
+    Path tmpDir = new Path(snapshotDir, ".tmp");
+    Path workingDir = new Path(tmpDir, "not_a_snapshot");
+    assertFalse("Already have working snapshot dir: " + workingDir
+        + " but shouldn't. Test file leak?", fs.exists(workingDir));
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot").build();
+    try {
+      SnapshotDescriptionUtils.completeSnapshot(snapshot, root, workingDir, fs);
+      fail("Shouldn't successfully complete move of a non-existent directory.");
+    } catch (IOException e) {
+      LOG.info("Correctly failed to move non-existant directory: " + e.getMessage());
+    }
+  }
+}
\ No newline at end of file