HBASE-15521 Procedure V2 - RestoreSnapshot and CloneSnapshot (Stephen Yuan 
Jiang)


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

Branch: refs/heads/HBASE-14850
Commit: e1d5c3d269129cf8c313ba29d9a9f694b799170b
Parents: ff6a339
Author: Stephen Yuan Jiang <syuanjiang...@gmail.com>
Authored: Thu Mar 31 21:49:13 2016 -0700
Committer: Stephen Yuan Jiang <syuanjiang...@gmail.com>
Committed: Thu Mar 31 21:49:13 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   36 +-
 .../hbase/client/ConnectionImplementation.java  |    7 -
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  199 +-
 .../hbase/procedure2/ProcedureExecutor.java     |    2 +-
 .../generated/MasterProcedureProtos.java        | 5804 +++++++++++++++++-
 .../hbase/protobuf/generated/MasterProtos.java  | 1040 ++--
 hbase-protocol/src/main/protobuf/Master.proto   |    8 +-
 .../src/main/protobuf/MasterProcedure.proto     |   40 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   32 +-
 .../procedure/CloneSnapshotProcedure.java       |  522 ++
 .../master/procedure/CreateTableProcedure.java  |   16 +-
 .../procedure/MasterDDLOperationHelper.java     |   13 +
 .../procedure/RestoreSnapshotProcedure.java     |  526 ++
 .../master/snapshot/CloneSnapshotHandler.java   |  195 -
 .../master/snapshot/RestoreSnapshotHandler.java |  245 -
 .../hbase/master/snapshot/SnapshotManager.java  |  330 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |   14 +-
 .../client/TestSnapshotCloneIndependence.java   |    1 -
 .../MasterProcedureTestingUtility.java          |    4 +-
 .../procedure/TestCloneSnapshotProcedure.java   |  239 +
 .../procedure/TestCreateTableProcedure.java     |    4 +-
 .../procedure/TestDeleteTableProcedure.java     |    8 +-
 .../TestMasterFailoverWithProcedures.java       |    2 +-
 .../master/procedure/TestProcedureAdmin.java    |    2 +-
 .../procedure/TestRestoreSnapshotProcedure.java |  291 +
 25 files changed, 8200 insertions(+), 1380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index c3b524b..54d2cb9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1330,6 +1330,22 @@ public interface Admin extends Abortable, Closeable {
   void restoreSnapshot(final String snapshotName) throws IOException, 
RestoreSnapshotException;
 
   /**
+   * Restore the specified snapshot on the original table. (The table must be 
disabled) If the
+   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is 
set to true, a
+   * snapshot of the current table is taken before executing the restore 
operation. In case of
+   * restore failure, the failsafe snapshot will be restored. If the restore 
completes without
+   * problem the failsafe snapshot is deleted.
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if snapshot failed to be restored
+   * @return the result of the async restore snapshot. You can use 
Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
+  Future<Void> restoreSnapshotAsync(final String snapshotName)
+      throws IOException, RestoreSnapshotException;
+
+  /**
    * Restore the specified snapshot on the original table. (The table must be 
disabled) If
    * 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is 
taken before
    * executing the restore operation. In case of restore failure, the failsafe 
snapshot will be
@@ -1360,7 +1376,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted 
incorrectly
    */
-  void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
+  void restoreSnapshot(final String snapshotName, final boolean 
takeFailSafeSnapshot)
       throws IOException, RestoreSnapshotException;
 
   /**
@@ -1390,6 +1406,24 @@ public interface Admin extends Abortable, Closeable {
       throws IOException, TableExistsException, RestoreSnapshotException;
 
   /**
+   * Create a new table by cloning the snapshot content, but does not block
+   * and wait for it be completely cloned.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to 
complete.
+   * It may throw ExecutionException if there was an error while executing the 
operation
+   * or TimeoutException in case the wait timeout was not long enough to allow 
the
+   * operation to complete.
+   *
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be cloned already exists
+   * @return the result of the async clone snapshot. You can use 
Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
+  Future<Void> cloneSnapshotAsync(final String snapshotName, final TableName 
tableName)
+      throws IOException, TableExistsException;
+
+  /**
    * Execute a distributed procedure on a cluster.
    *
    * @param signature A distributed procedure is uniquely identified by its 
signature (default the

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index ecaf18b..c71d81a 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1588,13 +1588,6 @@ class ConnectionImplementation implements 
ClusterConnection, Closeable {
       }
 
       @Override
-      public MasterProtos.IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(
-          RpcController controller, MasterProtos.IsRestoreSnapshotDoneRequest 
request)
-          throws ServiceException {
-        return stub.isRestoreSnapshotDone(controller, request);
-      }
-
-      @Override
       public MasterProtos.ExecProcedureResponse execProcedure(
           RpcController controller, MasterProtos.ExecProcedureRequest request)
           throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index c1d07ae..aea86b9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -127,8 +127,6 @@ import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescripto
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
@@ -693,47 +691,6 @@ public class HBaseAdmin implements Admin {
     get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
-  /**
-   * Wait for the table to be enabled and available
-   * If enabling the table exceeds the retry period, an exception is thrown.
-   * @param tableName name of the table
-   * @throws IOException if a remote or network exception occurs or
-   *    table is not enabled after the retries period.
-   */
-  private void waitUntilTableIsEnabled(final TableName tableName) throws 
IOException {
-    boolean enabled = false;
-    long start = EnvironmentEdgeManager.currentTime();
-    for (int tries = 0; tries < (this.numRetries * 
this.retryLongerMultiplier); tries++) {
-      try {
-        enabled = isTableEnabled(tableName);
-      } catch (TableNotFoundException tnfe) {
-        // wait for table to be created
-        enabled = false;
-      }
-      enabled = enabled && isTableAvailable(tableName);
-      if (enabled) {
-        break;
-      }
-      long sleep = getPauseTime(tries);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
-          "enabled in " + tableName);
-      }
-      try {
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        // Do this conversion rather than let it out because do not want to
-        // change the method signature.
-        throw (InterruptedIOException)new 
InterruptedIOException("Interrupted").initCause(e);
-      }
-    }
-    if (!enabled) {
-      long msec = EnvironmentEdgeManager.currentTime() - start;
-      throw new IOException("Table '" + tableName +
-        "' not yet enabled, after " + msec + "ms.");
-    }
-  }
-
   @Override
   public Future<Void> enableTableAsync(final TableName tableName) throws 
IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
@@ -2430,8 +2387,14 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
   }
 
-  @Override
-  public void restoreSnapshot(final String snapshotName, boolean 
takeFailSafeSnapshot)
+  /*
+   * Check whether the snapshot exists and contains disabled table
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if no valid snapshot is found
+   */
+  private TableName getTableNameBeforeRestoreSnapshot(final String 
snapshotName)
       throws IOException, RestoreSnapshotException {
     TableName tableName = null;
     for (SnapshotDescription snapshotInfo: listSnapshots()) {
@@ -2445,6 +2408,13 @@ public class HBaseAdmin implements Admin {
       throw new RestoreSnapshotException(
         "Unable to find the table name for snapshot=" + snapshotName);
     }
+    return tableName;
+  }
+
+  @Override
+  public void restoreSnapshot(final String snapshotName, final boolean 
takeFailSafeSnapshot)
+      throws IOException, RestoreSnapshotException {
+    TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
 
     // The table does not exists, switch to clone.
     if (!tableExists(tableName)) {
@@ -2472,13 +2442,19 @@ public class HBaseAdmin implements Admin {
 
     try {
       // Restore snapshot
-      internalRestoreSnapshot(snapshotName, tableName);
+      get(
+        internalRestoreSnapshotAsync(snapshotName, tableName),
+        syncWaitTimeout,
+        TimeUnit.MILLISECONDS);
     } catch (IOException e) {
       // Somthing went wrong during the restore...
       // if the pre-restore snapshot is available try to rollback
       if (takeFailSafeSnapshot) {
         try {
-          internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName);
+          get(
+            internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, 
tableName),
+            syncWaitTimeout,
+            TimeUnit.MILLISECONDS);
           String msg = "Restore snapshot=" + snapshotName +
             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + 
" succeeded.";
           LOG.error(msg, e);
@@ -2505,6 +2481,24 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public Future<Void> restoreSnapshotAsync(final String snapshotName)
+      throws IOException, RestoreSnapshotException {
+    TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
+
+    // The table does not exists, switch to clone.
+    if (!tableExists(tableName)) {
+      return cloneSnapshotAsync(snapshotName, tableName);
+    }
+
+    // Check if the table is disabled
+    if (!isTableDisabled(tableName)) {
+      throw new TableNotDisabledException(tableName);
+    }
+
+    return internalRestoreSnapshotAsync(snapshotName, tableName);
+  }
+
+  @Override
   public void cloneSnapshot(final byte[] snapshotName, final TableName 
tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
     cloneSnapshot(Bytes.toString(snapshotName), tableName);
@@ -2516,8 +2510,19 @@ public class HBaseAdmin implements Admin {
     if (tableExists(tableName)) {
       throw new TableExistsException(tableName);
     }
-    internalRestoreSnapshot(snapshotName, tableName);
-    waitUntilTableIsEnabled(tableName);
+    get(
+      internalRestoreSnapshotAsync(snapshotName, tableName),
+      Integer.MAX_VALUE,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> cloneSnapshotAsync(final String snapshotName, final 
TableName tableName)
+      throws IOException, TableExistsException {
+    if (tableExists(tableName)) {
+      throw new TableExistsException(tableName);
+    }
+    return internalRestoreSnapshotAsync(snapshotName, tableName);
   }
 
   @Override
@@ -2632,73 +2637,59 @@ public class HBaseAdmin implements Admin {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted 
incorrectly
    */
-  private void internalRestoreSnapshot(final String snapshotName, final 
TableName tableName)
-      throws IOException, RestoreSnapshotException {
-    SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+  private Future<Void> internalRestoreSnapshotAsync(
+      final String snapshotName,
+      final TableName tableName) throws IOException, RestoreSnapshotException {
+    final SnapshotDescription snapshot = SnapshotDescription.newBuilder()
         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
 
     // actually restore the snapshot
-    internalRestoreSnapshotAsync(snapshot);
-
-    final IsRestoreSnapshotDoneRequest request = 
IsRestoreSnapshotDoneRequest.newBuilder()
-        .setSnapshot(snapshot).build();
-    IsRestoreSnapshotDoneResponse done = 
IsRestoreSnapshotDoneResponse.newBuilder()
-        .setDone(false).buildPartial();
-    final long maxPauseTime = 5000;
-    int tries = 0;
-    while (!done.getDone()) {
-      try {
-        // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
-        sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
-        LOG.debug(tries + ") Sleeping: " + sleep
-            + " ms while we wait for snapshot restore to complete.");
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new 
InterruptedIOException("Interrupted").initCause(e);
-      }
-      LOG.debug("Getting current status of snapshot restore from master...");
-      done = executeCallable(new MasterCallable<IsRestoreSnapshotDoneResponse>(
-          getConnection()) {
-        @Override
-        public IsRestoreSnapshotDoneResponse call(int callTimeout) throws 
ServiceException {
-          PayloadCarryingRpcController controller = 
rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          return master.isRestoreSnapshotDone(controller, request);
-        }
-      });
-    }
-    if (!done.getDone()) {
-      throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + 
"' wasn't restored.");
-    }
-  }
-
-  /**
-   * Execute Restore/Clone snapshot and wait for the server to complete 
(asynchronous)
-   * <p>
-   * Only a single snapshot should be restored at a time, or results may be 
undefined.
-   * @param snapshot snapshot to restore
-   * @return response from the server indicating the max time to wait for the 
snapshot
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted 
incorrectly
-   */
-  private RestoreSnapshotResponse internalRestoreSnapshotAsync(final 
SnapshotDescription snapshot)
-      throws IOException, RestoreSnapshotException {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
 
-    final RestoreSnapshotRequest request = 
RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
-        .build();
-
-    // run the snapshot restore on the master
-    return executeCallable(new 
MasterCallable<RestoreSnapshotResponse>(getConnection()) {
+    RestoreSnapshotResponse response = executeCallable(
+        new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
       @Override
       public RestoreSnapshotResponse call(int callTimeout) throws 
ServiceException {
+        final RestoreSnapshotRequest request = 
RestoreSnapshotRequest.newBuilder()
+            .setSnapshot(snapshot)
+            .setNonceGroup(ng.getNonceGroup())
+            .setNonce(ng.newNonce())
+            .build();
         PayloadCarryingRpcController controller = 
rpcControllerFactory.newController();
         controller.setCallTimeout(callTimeout);
         return master.restoreSnapshot(controller, request);
       }
     });
+
+    return new RestoreSnapshotFuture(
+      this, snapshot, TableName.valueOf(snapshot.getTable()), response);
+  }
+
+  private static class RestoreSnapshotFuture extends TableFuture<Void> {
+    public RestoreSnapshotFuture(
+        final HBaseAdmin admin,
+        final SnapshotDescription snapshot,
+        final TableName tableName,
+        final RestoreSnapshotResponse response) {
+      super(admin, tableName,
+          (response != null && response.hasProcId()) ? response.getProcId() : 
null);
+
+      if (response != null && !response.hasProcId()) {
+        throw new UnsupportedOperationException("Client could not call old 
version of Server");
+      }
+    }
+
+    public RestoreSnapshotFuture(
+        final HBaseAdmin admin,
+        final TableName tableName,
+        final Long procId) {
+      super(admin, tableName, procId);
+    }
+
+    @Override
+    public String getOperationType() {
+      return "MODIFY";
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d5c3d2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 38dd062..3f0ba37 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -694,7 +694,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @return true if the procedure execution is finished, otherwise false.
    */
   public boolean isFinished(final long procId) {
-    return completed.containsKey(procId);
+    return !procedures.containsKey(procId);
   }
 
   /**

Reply via email to