http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index db718ee..50a1832 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -56,16 +56,19 @@ import org.apache.hadoop.hbase.master.MetricsMaster;
 import org.apache.hadoop.hbase.master.SnapshotSentinel;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
+import org.apache.hadoop.hbase.master.procedure.CloneSnapshotProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RestoreSnapshotProcedure;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure.Procedure;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
-import org.apache.hadoop.hbase.quotas.QuotaExceededException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -145,12 +148,14 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
   private Map<TableName, SnapshotSentinel> snapshotHandlers =
       new HashMap<TableName, SnapshotSentinel>();
 
-  // Restore Sentinels map, with table name as key.
+  // Restore map, with table name as key, procedure ID as value.
   // The map is always accessed and modified under the object lock using 
synchronized.
-  // restoreSnapshot()/cloneSnapshot() will insert an Handler in the table.
-  // isRestoreDone() will remove the handler requested if the operation is 
finished.
-  private Map<TableName, SnapshotSentinel> restoreHandlers =
-      new HashMap<TableName, SnapshotSentinel>();
+  // restoreSnapshot()/cloneSnapshot() will insert a procedure ID in the map.
+  //
+  // TODO: just as the Apache HBase 1.x implementation, this map would not 
survive master
+  // restart/failover. This is just a stopgap implementation until 
implementation of taking
+  // snapshot using Procedure-V2.
+  private Map<TableName, Long> restoreTableToProcIdMap = new 
HashMap<TableName, Long>();
 
   private Path rootDir;
   private ExecutorService executorService;
@@ -426,11 +431,9 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
 
     // make sure we aren't running a restore on the same table
     if (isRestoringTable(snapshotTable)) {
-      SnapshotSentinel handler = restoreHandlers.get(snapshotTable);
       throw new SnapshotCreationException("Rejected taking "
           + ClientSnapshotDescriptionUtils.toString(snapshot)
-          + " because we are already have a restore in progress on the same 
snapshot "
-          + ClientSnapshotDescriptionUtils.toString(handler.getSnapshot()), 
snapshot);
+          + " because we are already have a restore in progress on the same 
snapshot.");
     }
 
     try {
@@ -647,14 +650,61 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
   }
 
   /**
+   * Clone the specified snapshot.
+   * The clone will fail if the destination table has a snapshot or restore in 
progress.
+   *
+   * @param reqSnapshot Snapshot Descriptor from request
+   * @param tableName table to clone
+   * @param snapshot Snapshot Descriptor
+   * @param snapshotTableDesc Table Descriptor
+   * @param nonceGroup unique value to prevent duplicated RPC
+   * @param nonce unique value to prevent duplicated RPC
+   * @return procId the ID of the clone snapshot procedure
+   * @throws IOException
+   */
+  private long cloneSnapshot(
+      final SnapshotDescription reqSnapshot,
+      final TableName tableName,
+      final SnapshotDescription snapshot,
+      final HTableDescriptor snapshotTableDesc,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
+    HTableDescriptor htd = new HTableDescriptor(tableName, snapshotTableDesc);
+    if (cpHost != null) {
+      cpHost.preCloneSnapshot(reqSnapshot, htd);
+    }
+    long procId;
+    try {
+      procId = cloneSnapshot(snapshot, htd, nonceGroup, nonce);
+    } catch (IOException e) {
+      LOG.error("Exception occurred while cloning the snapshot " + 
snapshot.getName()
+        + " as table " + tableName.getNameAsString(), e);
+      throw e;
+    }
+    LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + 
tableName);
+
+    if (cpHost != null) {
+      cpHost.postCloneSnapshot(reqSnapshot, htd);
+    }
+    return procId;
+  }
+
+  /**
    * Clone the specified snapshot into a new table.
    * The operation will fail if the destination table has a snapshot or 
restore in progress.
    *
    * @param snapshot Snapshot Descriptor
    * @param hTableDescriptor Table Descriptor of the table to create
+   * @param nonceGroup unique value to prevent duplicated RPC
+   * @param nonce unique value to prevent duplicated RPC
+   * @return procId the ID of the clone snapshot procedure
    */
-  synchronized void cloneSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
+  synchronized long cloneSnapshot(
+      final SnapshotDescription snapshot,
+      final HTableDescriptor hTableDescriptor,
+      final long nonceGroup,
+      final long nonce) throws HBaseSnapshotException {
     TableName tableName = hTableDescriptor.getTableName();
 
     // make sure we aren't running a snapshot on the same table
@@ -668,27 +718,34 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
     }
 
     try {
-      CloneSnapshotHandler handler =
-        new CloneSnapshotHandler(master, snapshot, hTableDescriptor).prepare();
-      this.executorService.submit(handler);
-      this.restoreHandlers.put(tableName, handler);
+      long procId = master.getMasterProcedureExecutor().submitProcedure(
+        new CloneSnapshotProcedure(
+          master.getMasterProcedureExecutor().getEnvironment(), 
hTableDescriptor, snapshot),
+        nonceGroup,
+        nonce);
+      this.restoreTableToProcIdMap.put(tableName, procId);
+      return procId;
     } catch (Exception e) {
-      String msg = "Couldn't clone the snapshot=" + 
ClientSnapshotDescriptionUtils.toString(snapshot) +
-        " on table=" + tableName;
+      String msg = "Couldn't clone the snapshot="
+        + ClientSnapshotDescriptionUtils.toString(snapshot) + " on table=" + 
tableName;
       LOG.error(msg, e);
       throw new RestoreSnapshotException(msg, e);
     }
   }
 
   /**
-   * Restore the specified snapshot
+   * Restore or Clone the specified snapshot
    * @param reqSnapshot
+   * @param nonceGroup unique value to prevent duplicated RPC
+   * @param nonce unique value to prevent duplicated RPC
    * @throws IOException
    */
-  public void restoreSnapshot(SnapshotDescription reqSnapshot) throws 
IOException {
+  public long restoreOrCloneSnapshot(
+      SnapshotDescription reqSnapshot,
+      final long nonceGroup,
+      final long nonce) throws IOException {
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
     Path snapshotDir = 
SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
-    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
 
     // check if the snapshot exists
     if (!fs.exists(snapshotDir)) {
@@ -712,109 +769,66 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
     SnapshotReferenceUtil.verifySnapshot(master.getConfiguration(), fs, 
manifest);
 
     // Execute the restore/clone operation
+    long procId;
     if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
-      if (master.getTableStateManager().isTableState(
-          TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
-        throw new UnsupportedOperationException("Table '" +
-            TableName.valueOf(snapshot.getTable()) + "' must be disabled in 
order to " +
-            "perform a restore operation" +
-            ".");
-      }
-
-      // call coproc pre hook
-      if (cpHost != null) {
-        cpHost.preRestoreSnapshot(reqSnapshot, snapshotTableDesc);
-      }
-
-      int tableRegionCount = -1;
-      try {
-        // Table already exist. Check and update the region quota for this 
table namespace.
-        // The region quota may not be updated correctly if there are 
concurrent restore snapshot
-        // requests for the same table
-
-        tableRegionCount = getRegionCountOfTable(tableName);
-        int snapshotRegionCount = manifest.getRegionManifestsMap().size();
-
-        // Update region quota when snapshotRegionCount is larger. If we 
updated the region count
-        // to a smaller value before retoreSnapshot and the retoreSnapshot 
fails, we may fail to
-        // reset the region count to its original value if the region quota is 
consumed by other
-        // tables in the namespace
-        if (tableRegionCount > 0 && tableRegionCount < snapshotRegionCount) {
-          checkAndUpdateNamespaceRegionQuota(snapshotRegionCount, tableName);
-        }
-        restoreSnapshot(snapshot, snapshotTableDesc);
-        // Update the region quota if snapshotRegionCount is smaller. This 
step should not fail
-        // because we have reserved enough region quota before hand
-        if (tableRegionCount > 0 && tableRegionCount > snapshotRegionCount) {
-          checkAndUpdateNamespaceRegionQuota(snapshotRegionCount, tableName);
-        }
-      } catch (QuotaExceededException e) {
-        LOG.error("Region quota exceeded while restoring the snapshot " + 
snapshot.getName()
-          + " as table " + tableName.getNameAsString(), e);
-        // If QEE is thrown before restoreSnapshot, quota information is not 
updated, so we
-        // should throw the exception directly. If QEE is thrown after 
restoreSnapshot, there
-        // must be unexpected reasons, we also throw the exception directly
-        throw e;
-      } catch (IOException e) {
-        if (tableRegionCount > 0) {
-          // reset the region count for table
-          checkAndUpdateNamespaceRegionQuota(tableRegionCount, tableName);
-        }
-        LOG.error("Exception occurred while restoring the snapshot " + 
snapshot.getName()
-            + " as table " + tableName.getNameAsString(), e);
-        throw e;
-      }
-      LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + 
tableName);
-
-      if (cpHost != null) {
-        cpHost.postRestoreSnapshot(reqSnapshot, snapshotTableDesc);
-      }
+      procId = restoreSnapshot(
+        reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceGroup, 
nonce);
     } else {
-      HTableDescriptor htd = new HTableDescriptor(tableName, 
snapshotTableDesc);
-      if (cpHost != null) {
-        cpHost.preCloneSnapshot(reqSnapshot, htd);
-      }
-      try {
-        checkAndUpdateNamespaceQuota(manifest, tableName);
-        cloneSnapshot(snapshot, htd);
-      } catch (IOException e) {
-        
this.master.getMasterQuotaManager().removeTableFromNamespaceQuota(tableName);
-        LOG.error("Exception occurred while cloning the snapshot " + 
snapshot.getName()
-            + " as table " + tableName.getNameAsString(), e);
-        throw e;
-      }
-      LOG.info("Clone snapshot=" + snapshot.getName() + " as table=" + 
tableName);
-
-      if (cpHost != null) {
-        cpHost.postCloneSnapshot(reqSnapshot, htd);
-      }
+      procId = cloneSnapshot(
+        reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceGroup, 
nonce);
     }
+    return procId;
   }
 
-  private void checkAndUpdateNamespaceQuota(SnapshotManifest manifest, 
TableName tableName)
-      throws IOException {
-    if (this.master.getMasterQuotaManager().isQuotaEnabled()) {
-      
this.master.getMasterQuotaManager().checkNamespaceTableAndRegionQuota(tableName,
-        manifest.getRegionManifestsMap().size());
+  /**
+   * Restore the specified snapshot.
+   * The restore will fail if the destination table has a snapshot or restore 
in progress.
+   *
+   * @param reqSnapshot Snapshot Descriptor from request
+   * @param tableName table to restore
+   * @param snapshot Snapshot Descriptor
+   * @param snapshotTableDesc Table Descriptor
+   * @param nonceGroup unique value to prevent duplicated RPC
+   * @param nonce unique value to prevent duplicated RPC
+   * @return procId the ID of the restore snapshot procedure
+   * @throws IOException
+   */
+  private long restoreSnapshot(
+      final SnapshotDescription reqSnapshot,
+      final TableName tableName,
+      final SnapshotDescription snapshot,
+      final HTableDescriptor snapshotTableDesc,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
+
+    if (master.getTableStateManager().isTableState(
+      TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
+      throw new UnsupportedOperationException("Table '" +
+        TableName.valueOf(snapshot.getTable()) + "' must be disabled in order 
to " +
+        "perform a restore operation.");
     }
-  }
 
-  private void checkAndUpdateNamespaceRegionQuota(int updatedRegionCount, 
TableName tableName)
-      throws IOException {
-    if (this.master.getMasterQuotaManager().isQuotaEnabled()) {
-      
this.master.getMasterQuotaManager().checkAndUpdateNamespaceRegionQuota(tableName,
-        updatedRegionCount);
+    // call Coprocessor pre hook
+    if (cpHost != null) {
+      cpHost.preRestoreSnapshot(reqSnapshot, snapshotTableDesc);
     }
-  }
 
-  /**
-   * @return cached region count, or -1 if quota manager is disabled or table 
status not found
-  */
-  private int getRegionCountOfTable(TableName tableName) throws IOException {
-    if (this.master.getMasterQuotaManager().isQuotaEnabled()) {
-      return 
this.master.getMasterQuotaManager().getRegionCountOfTable(tableName);
+    long procId;
+    try {
+      procId = restoreSnapshot(snapshot, snapshotTableDesc, nonceGroup, nonce);
+    } catch (IOException e) {
+      LOG.error("Exception occurred while restoring the snapshot " + 
snapshot.getName()
+        + " as table " + tableName.getNameAsString(), e);
+      throw e;
+    }
+    LOG.info("Restore snapshot=" + snapshot.getName() + " as table=" + 
tableName);
+
+    if (cpHost != null) {
+      cpHost.postRestoreSnapshot(reqSnapshot, snapshotTableDesc);
     }
-    return -1;
+
+    return procId;
   }
 
   /**
@@ -823,9 +837,15 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
    *
    * @param snapshot Snapshot Descriptor
    * @param hTableDescriptor Table Descriptor
+   * @param nonceGroup unique value to prevent duplicated RPC
+   * @param nonce unique value to prevent duplicated RPC
+   * @return procId the ID of the restore snapshot procedure
    */
-  private synchronized void restoreSnapshot(final SnapshotDescription snapshot,
-      final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
+  private synchronized long restoreSnapshot(
+      final SnapshotDescription snapshot,
+      final HTableDescriptor hTableDescriptor,
+      final long nonceGroup,
+      final long nonce) throws HBaseSnapshotException {
     TableName tableName = hTableDescriptor.getTableName();
 
     // make sure we aren't running a snapshot on the same table
@@ -839,10 +859,13 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
     }
 
     try {
-      RestoreSnapshotHandler handler =
-        new RestoreSnapshotHandler(master, snapshot, 
hTableDescriptor).prepare();
-      this.executorService.submit(handler);
-      restoreHandlers.put(tableName, handler);
+      long procId = master.getMasterProcedureExecutor().submitProcedure(
+        new RestoreSnapshotProcedure(
+          master.getMasterProcedureExecutor().getEnvironment(), 
hTableDescriptor, snapshot),
+        nonceGroup,
+        nonce);
+      this.restoreTableToProcIdMap.put(tableName, procId);
+      return procId;
     } catch (Exception e) {
       String msg = "Couldn't restore the snapshot=" + 
ClientSnapshotDescriptionUtils.toString(
           snapshot)  +
@@ -859,50 +882,18 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
    * @return <tt>true</tt> if there is a restore in progress of the specified 
table.
    */
   private synchronized boolean isRestoringTable(final TableName tableName) {
-    SnapshotSentinel sentinel = this.restoreHandlers.get(tableName);
-    return(sentinel != null && !sentinel.isFinished());
-  }
-
-  /**
-   * Returns the status of a restore operation.
-   * If the in-progress restore is failed throws the exception that caused the 
failure.
-   *
-   * @param snapshot
-   * @return false if in progress, true if restore is completed or not 
requested.
-   * @throws IOException if there was a failure during the restore
-   */
-  public boolean isRestoreDone(final SnapshotDescription snapshot) throws 
IOException {
-    // check to see if the sentinel exists,
-    // and if the task is complete removes it from the in-progress restore map.
-    SnapshotSentinel sentinel = removeSentinelIfFinished(this.restoreHandlers, 
snapshot);
-
-    // stop tracking "abandoned" handlers
-    cleanupSentinels();
-
-    if (sentinel == null) {
-      // there is no sentinel so restore is not in progress.
-      return true;
+    Long procId = this.restoreTableToProcIdMap.get(tableName);
+    if (procId == null) {
+      return false;
     }
-
-    LOG.debug("Verify snapshot=" + snapshot.getName() + " against="
-        + sentinel.getSnapshot().getName() + " table=" +
-        TableName.valueOf(snapshot.getTable()));
-
-    // If the restore is failed, rethrow the exception
-    sentinel.rethrowExceptionIfFailed();
-
-    // check to see if we are done
-    if (sentinel.isFinished()) {
-      LOG.debug("Restore snapshot=" + 
ClientSnapshotDescriptionUtils.toString(snapshot) +
-          " has completed. Notifying the client.");
+    ProcedureExecutor<MasterProcedureEnv> procExec = 
master.getMasterProcedureExecutor();
+    if (procExec.isRunning() && !procExec.isFinished(procId)) {
       return true;
+    } else {
+      this.restoreTableToProcIdMap.remove(tableName);
+      return false;
     }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Sentinel is not yet finished with restoring snapshot=" +
-          ClientSnapshotDescriptionUtils.toString(snapshot));
-    }
-    return false;
   }
 
   /**
@@ -947,7 +938,7 @@ public class SnapshotManager extends MasterProcedureManager 
implements Stoppable
    */
   private void cleanupSentinels() {
     cleanupSentinels(this.snapshotHandlers);
-    cleanupSentinels(this.restoreHandlers);
+    cleanupCompletedRestoreInMap();
   }
 
   /**
@@ -970,6 +961,21 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
     }
   }
 
+  /**
+   * Remove the procedures that are marked as finished
+   */
+  private synchronized void cleanupCompletedRestoreInMap() {
+    ProcedureExecutor<MasterProcedureEnv> procExec = 
master.getMasterProcedureExecutor();
+    Iterator<Map.Entry<TableName, Long>> it = 
restoreTableToProcIdMap.entrySet().iterator();
+    while (it.hasNext()) {
+      Map.Entry<TableName, Long> entry = it.next();
+      Long procId = entry.getValue();
+      if (procExec.isRunning() && procExec.isFinished(procId)) {
+        it.remove();
+      }
+    }
+  }
+
   //
   // Implementing Stoppable interface
   //
@@ -985,10 +991,6 @@ public class SnapshotManager extends 
MasterProcedureManager implements Stoppable
       snapshotHandler.cancel(why);
     }
 
-    // pass the stop onto all the restore handlers
-    for (SnapshotSentinel restoreHandler: this.restoreHandlers.values()) {
-      restoreHandler.cancel(why);
-    }
     try {
       if (coordinator != null) {
         coordinator.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 0f14f70..a8542de 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -175,7 +175,7 @@ public class RestoreSnapshotHelper {
   }
 
   private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) 
throws IOException {
-    LOG.debug("starting restore");
+    LOG.info("starting restore table regions using snapshot=" + snapshotDesc);
 
     Map<String, SnapshotRegionManifest> regionManifests = 
snapshotManifest.getRegionManifestsMap();
     if (regionManifests == null) {
@@ -251,6 +251,8 @@ public class RestoreSnapshotHelper {
       status.setStatus("Finished cloning regions.");
     }
 
+    LOG.info("finishing restore table regions using snapshot=" + snapshotDesc);
+
     return metaChanges;
   }
 
@@ -265,7 +267,7 @@ public class RestoreSnapshotHelper {
     private List<HRegionInfo> regionsToRemove = null;
     private List<HRegionInfo> regionsToAdd = null;
 
-    RestoreMetaChanges(HTableDescriptor htd, Map<String, Pair<String, String> 
> parentsMap) {
+    public RestoreMetaChanges(HTableDescriptor htd, Map<String, Pair<String, 
String> > parentsMap) {
       this.parentsMap = parentsMap;
       this.htd = htd;
     }
@@ -275,6 +277,14 @@ public class RestoreSnapshotHelper {
     }
 
     /**
+     * Returns the map of parent-children_pair.
+     * @return the map
+     */
+    public Map<String, Pair<String, String>> getParentToChildrenPairMap() {
+      return this.parentsMap;
+    }
+
+    /**
      * @return true if there're new regions
      */
     public boolean hasRegionsToAdd() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 0a3093b..002e04e 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -44,7 +44,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 9731aa4..430a017 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -129,8 +129,8 @@ public class MasterProcedureTestingUtility {
     assertEquals(family.length, htd.getFamilies().size());
   }
 
-  public static void validateTableDeletion(final HMaster master, final 
TableName tableName,
-      final HRegionInfo[] regions, String... family) throws IOException {
+  public static void validateTableDeletion(
+      final HMaster master, final TableName tableName) throws IOException {
     // check filesystem
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
     final Path tableDir = 
FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
new file mode 100644
index 0000000..aeafbf8
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
@@ -0,0 +1,239 @@
+/**
+ * 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.procedure;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import 
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestCloneSnapshotProcedure {
+  private static final Log LOG = 
LogFactory.getLog(TestCloneSnapshotProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  protected final byte[] CF = Bytes.toBytes("cf1");
+
+  private static long nonceGroup = HConstants.NO_NONCE;
+  private static long nonce = HConstants.NO_NONCE;
+
+  private static SnapshotDescription snapshot = null;
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    resetProcExecutorTestingKillFlag();
+    nonceGroup =
+        
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+    nonce = 
MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    resetProcExecutorTestingKillFlag();
+  }
+
+  private void resetProcExecutorTestingKillFlag() {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  private SnapshotDescription getSnapshot() throws Exception {
+    if (snapshot == null) {
+      final TableName snapshotTableName = 
TableName.valueOf("testCloneSnapshot");
+      long tid = System.currentTimeMillis();
+      final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+
+      Admin admin = UTIL.getHBaseAdmin();
+      // create Table
+      SnapshotTestingUtils.createTable(UTIL, snapshotTableName, 
getNumReplicas(), CF);
+      // Load data
+      SnapshotTestingUtils.loadData(UTIL, snapshotTableName, 500, CF);
+      admin.disableTable(snapshotTableName);
+      // take a snapshot
+      admin.snapshot(snapshotName, snapshotTableName);
+      admin.enableTable(snapshotTableName);
+
+      List<SnapshotDescription> snapshotList = admin.listSnapshots();
+      snapshot = snapshotList.get(0);
+    }
+    return snapshot;
+  }
+
+  private int getNumReplicas() {
+    return 1;
+  }
+
+  public static HTableDescriptor createHTableDescriptor(
+      final TableName tableName, final byte[] ... family) {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    for (int i = 0; i < family.length; ++i) {
+      htd.addFamily(new HColumnDescriptor(family[i]));
+    }
+    return htd;
+  }
+
+  @Test(timeout=60000)
+  public void testCloneSnapshot() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName clonedTableName = TableName.valueOf("testCloneSnapshot2");
+    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+      procExec, new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+    MasterProcedureTestingUtility.validateTableIsEnabled(
+      UTIL.getHBaseCluster().getMaster(),
+      clonedTableName);
+  }
+
+  @Test(timeout = 60000)
+  public void testCloneSnapshotTwiceWithSameNonce() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName clonedTableName = 
TableName.valueOf("testCloneSnapshotTwiceWithSameNonce");
+    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+
+    long procId1 = procExec.submitProcedure(
+      new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()), nonceGroup, nonce);
+    long procId2 = procExec.submitProcedure(
+      new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()), nonceGroup, nonce);
+
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    // The second proc should succeed too - because it is the same proc.
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+    assertTrue(procId1 == procId2);
+  }
+
+  @Test(timeout=60000)
+  public void testCloneSnapshotToSameTable() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName clonedTableName = 
TableName.valueOf(getSnapshot().getTable());
+    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+      procExec, new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()));
+    ProcedureInfo result = procExec.getResult(procId);
+    assertTrue(result.isFailed());
+    LOG.debug("Clone snapshot failed with exception: " + 
result.getExceptionFullMessage());
+    assertTrue(
+      ProcedureTestingUtility.getExceptionCause(result) instanceof 
TableExistsException);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName clonedTableName = 
TableName.valueOf("testRecoveryAndDoubleExecution");
+    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Clone snapshot procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()), nonceGroup, nonce);
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = CloneSnapshotState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      CloneSnapshotState.values());
+
+    MasterProcedureTestingUtility.validateTableIsEnabled(
+      UTIL.getHBaseCluster().getMaster(),
+      clonedTableName);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName clonedTableName = 
TableName.valueOf("testRollbackAndDoubleExecution");
+    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Clone snapshot procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new CloneSnapshotProcedure(procExec.getEnvironment(), htd, 
getSnapshot()), nonceGroup, nonce);
+
+    int numberOfSteps = CloneSnapshotState.values().length - 2; // failing in 
the middle of proc
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      CloneSnapshotState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), clonedTableName);
+
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
index 8f6e11a..5cec469 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
@@ -253,7 +253,7 @@ public class TestCreateTableProcedure {
         procExec, procId, 4, CreateTableState.values());
 
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+      UTIL.getHBaseCluster().getMaster(), tableName);
 
     // are we able to create the table after a rollback?
     resetProcExecutorTestingKillFlag();
@@ -310,7 +310,7 @@ public class TestCreateTableProcedure {
         procExec, procId, 4, CreateTableState.values());
     TableName tableName = htd.getTableName();
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+      UTIL.getHBaseCluster().getMaster(), tableName);
 
     // are we able to create the table after a rollback?
     resetProcExecutorTestingKillFlag();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
index 4f62537..7eb12cd 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
@@ -139,7 +139,7 @@ public class TestDeleteTableProcedure {
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
+      UTIL.getHBaseCluster().getMaster(), tableName);
 
     // Second delete should fail with TableNotFound
     ProcedureInfo result = procExec.getResult(procId2);
@@ -171,7 +171,7 @@ public class TestDeleteTableProcedure {
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
+      UTIL.getHBaseCluster().getMaster(), tableName);
 
     // Second delete should not fail, because it is the same delete
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
@@ -205,7 +205,7 @@ public class TestDeleteTableProcedure {
       new DeleteTableProcedure(procExec.getEnvironment(), tableName));
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+      UTIL.getHBaseCluster().getMaster(), tableName);
   }
 
   @Test(timeout=60000)
@@ -233,7 +233,7 @@ public class TestDeleteTableProcedure {
       procExec, procId, 6, DeleteTableState.values());
 
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+      UTIL.getHBaseCluster().getMaster(), tableName);
   }
 
   private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index e880cd1..6098d40 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -301,7 +301,7 @@ public class TestMasterFailoverWithProcedures {
     testRecoveryAndDoubleExecution(UTIL, procId, step, 
DeleteTableState.values());
 
     MasterProcedureTestingUtility.validateTableDeletion(
-        UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+        UTIL.getHBaseCluster().getMaster(), tableName);
   }
 
   // ==========================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 49143f1..17b6e09 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -142,7 +142,7 @@ public class TestProcedureAdmin {
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
     // Validate the delete table procedure was not aborted
     MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
+      UTIL.getHBaseCluster().getMaster(), tableName);
   }
 
   @Test(timeout=60000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
new file mode 100644
index 0000000..44d6988
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
@@ -0,0 +1,291 @@
+/**
+ * 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.procedure;
+
+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.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import 
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestRestoreSnapshotProcedure {
+  private static final Log LOG = 
LogFactory.getLog(TestRestoreSnapshotProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  protected final TableName snapshotTableName = 
TableName.valueOf("testRestoreSnapshot");
+  protected final byte[] CF1 = Bytes.toBytes("cf1");
+  protected final byte[] CF2 = Bytes.toBytes("cf2");
+  protected final byte[] CF3 = Bytes.toBytes("cf3");
+  protected final byte[] CF4 = Bytes.toBytes("cf4");
+  protected final int rowCountCF1 = 10;
+  protected final int rowCountCF2 = 40;
+  protected final int rowCountCF3 = 40;
+  protected final int rowCountCF4 = 40;
+  protected final int rowCountCF1addition = 10;
+
+  private static long nonceGroup = HConstants.NO_NONCE;
+  private static long nonce = HConstants.NO_NONCE;
+
+  private SnapshotDescription snapshot = null;
+  private HTableDescriptor snapshotHTD = null;
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    resetProcExecutorTestingKillFlag();
+    nonceGroup =
+        
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+    nonce = 
MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+
+    setupSnapshotAndUpdateTable();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    resetProcExecutorTestingKillFlag();
+    UTIL.deleteTable(snapshotTableName);
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
+  }
+
+  private void resetProcExecutorTestingKillFlag() {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  private int getNumReplicas() {
+    return 1;
+  }
+
+  private void setupSnapshotAndUpdateTable() throws Exception {
+    long tid = System.currentTimeMillis();
+    final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+    Admin admin = UTIL.getHBaseAdmin();
+    // create Table
+    SnapshotTestingUtils.createTable(UTIL, snapshotTableName, 
getNumReplicas(), CF1, CF2);
+    // Load data
+    SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1, CF1);
+    SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF2, CF2);
+    SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + 
rowCountCF2);
+
+    snapshotHTD = admin.getTableDescriptor(snapshotTableName);
+
+    admin.disableTable(snapshotTableName);
+    // take a snapshot
+    admin.snapshot(snapshotName, snapshotTableName);
+
+    List<SnapshotDescription> snapshotList = admin.listSnapshots();
+    snapshot = snapshotList.get(0);
+
+    // modify the table
+    HColumnDescriptor columnFamilyDescriptor3 = new HColumnDescriptor(CF3);
+    HColumnDescriptor columnFamilyDescriptor4 = new HColumnDescriptor(CF4);
+    admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor3);
+    admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor4);
+    admin.deleteColumnFamily(snapshotTableName, CF2);
+    // enable table and insert data
+    admin.enableTable(snapshotTableName);
+    SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
+    SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
+    SnapshotTestingUtils.loadData(UTIL, snapshotTableName, 
rowCountCF1addition, CF1);
+    HTableDescriptor currentHTD = admin.getTableDescriptor(snapshotTableName);
+    assertTrue(currentHTD.hasFamily(CF1));
+    assertFalse(currentHTD.hasFamily(CF2));
+    assertTrue(currentHTD.hasFamily(CF3));
+    assertTrue(currentHTD.hasFamily(CF4));
+    assertNotEquals(currentHTD.getFamiliesKeys().size(), 
snapshotHTD.getFamiliesKeys().size());
+    SnapshotTestingUtils.verifyRowCount(
+      UTIL, snapshotTableName, rowCountCF1 + rowCountCF3 + rowCountCF4 + 
rowCountCF1addition);
+    admin.disableTable(snapshotTableName);
+  }
+
+  private static HTableDescriptor createHTableDescriptor(
+      final TableName tableName, final byte[] ... family) {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    for (int i = 0; i < family.length; ++i) {
+      htd.addFamily(new HColumnDescriptor(family[i]));
+    }
+    return htd;
+  }
+
+  @Test(timeout=600000)
+  public void testRestoreSnapshot() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+      procExec,
+      new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, 
snapshot));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+
+    validateSnapshotRestore();
+  }
+
+  @Test(timeout = 60000)
+  public void testRestoreSnapshotTwiceWithSameNonce() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+
+    long procId1 = procExec.submitProcedure(
+      new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, 
snapshot),
+      nonceGroup,
+      nonce);
+    long procId2 = procExec.submitProcedure(
+      new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, 
snapshot),
+      nonceGroup,
+      nonce);
+
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    // The second proc should succeed too - because it is the same proc.
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+    assertTrue(procId1 == procId2);
+
+    validateSnapshotRestore();
+  }
+
+  @Test(timeout=60000)
+  public void testRestoreSnapshotToDifferentTable() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+    final TableName restoredTableName = 
TableName.valueOf("testRestoreSnapshotToDifferentTable");
+    final HTableDescriptor newHTD = createHTableDescriptor(restoredTableName, 
CF1, CF2);
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+      procExec, new RestoreSnapshotProcedure(procExec.getEnvironment(), 
newHTD, snapshot));
+    ProcedureInfo result = procExec.getResult(procId);
+    assertTrue(result.isFailed());
+    LOG.debug("Restore snapshot failed with exception: " + 
result.getExceptionFullMessage());
+    assertTrue(
+      ProcedureTestingUtility.getExceptionCause(result) instanceof 
TableNotFoundException);
+  }
+
+  @Test(timeout=60000)
+  public void testRestoreSnapshotToEnabledTable() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+
+    try {
+      UTIL.getHBaseAdmin().enableTable(snapshotTableName);
+
+      long procId = ProcedureTestingUtility.submitAndWait(
+        procExec,
+        new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, 
snapshot));
+      ProcedureInfo result = procExec.getResult(procId);
+      assertTrue(result.isFailed());
+      LOG.debug("Restore snapshot failed with exception: " + 
result.getExceptionFullMessage());
+      assertTrue(
+        ProcedureTestingUtility.getExceptionCause(result) instanceof 
TableNotDisabledException);
+    } finally {
+      UTIL.getHBaseAdmin().disableTable(snapshotTableName);
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = 
getMasterProcedureExecutor();
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Restore snapshot procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, 
snapshot),
+      nonceGroup,
+      nonce);
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = RestoreSnapshotState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      RestoreSnapshotState.values());
+
+    resetProcExecutorTestingKillFlag();
+    validateSnapshotRestore();
+  }
+
+  private void validateSnapshotRestore() throws IOException {
+    try {
+      UTIL.getHBaseAdmin().enableTable(snapshotTableName);
+
+      HTableDescriptor currentHTD = 
UTIL.getHBaseAdmin().getTableDescriptor(snapshotTableName);
+      assertTrue(currentHTD.hasFamily(CF1));
+      assertTrue(currentHTD.hasFamily(CF2));
+      assertFalse(currentHTD.hasFamily(CF3));
+      assertFalse(currentHTD.hasFamily(CF4));
+      assertEquals(currentHTD.getFamiliesKeys().size(), 
snapshotHTD.getFamiliesKeys().size());
+      SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 
+ rowCountCF2);
+    } finally {
+      UTIL.getHBaseAdmin().disableTable(snapshotTableName);
+    }
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

Reply via email to