You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2017/05/18 09:56:10 UTC

[4/4] hbase git commit: HBASE-11013 Clone Snapshots on Secure Cluster Should provide option to apply Retained User Permissions

HBASE-11013 Clone Snapshots on Secure Cluster Should provide option to apply Retained User Permissions

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/branch-1
Commit: f9dc4cad63b1ffcd1f9050b9b8e8d89f44ecd44a
Parents: 68d292d
Author: huzheng <op...@gmail.com>
Authored: Mon May 15 16:00:01 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu May 18 17:54:53 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   49 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   63 +-
 .../hbase/security/access/TablePermission.java  |    4 +
 .../ClientSnapshotDescriptionUtils.java         |    8 +-
 .../snapshot/CorruptedSnapshotException.java    |    2 +-
 .../hbase/snapshot/HBaseSnapshotException.java  |    2 +-
 .../snapshot/RestoreSnapshotException.java      |    2 +-
 .../snapshot/SnapshotCreationException.java     |    2 +-
 .../snapshot/SnapshotDoesNotExistException.java |    2 +-
 .../hbase/snapshot/SnapshotExistsException.java |    2 +-
 .../hbase/client/TestSnapshotFromAdmin.java     |    2 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 1358 +--------------
 .../hbase/protobuf/generated/MasterProtos.java  | 1088 ++++++------
 .../protobuf/generated/SnapshotProtos.java      | 1551 +++++++++++++++++-
 hbase-protocol/src/main/protobuf/HBase.proto    |   17 -
 hbase-protocol/src/main/protobuf/Master.proto   |    2 +
 hbase-protocol/src/main/protobuf/Snapshot.proto |   19 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |    2 +-
 .../BaseMasterAndRegionObserver.java            |    2 +-
 .../hbase/coprocessor/BaseMasterObserver.java   |    2 +-
 .../hbase/coprocessor/MasterObserver.java       |    2 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java |    2 +-
 .../hbase/master/MasterCoprocessorHost.java     |    2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |    5 +-
 .../hadoop/hbase/master/SnapshotSentinel.java   |    2 +-
 .../master/snapshot/CloneSnapshotHandler.java   |   14 +-
 .../snapshot/DisabledTableSnapshotHandler.java  |    4 +-
 .../snapshot/EnabledTableSnapshotHandler.java   |    4 +-
 .../master/snapshot/MasterSnapshotVerifier.java |    2 +-
 .../master/snapshot/RestoreSnapshotHandler.java |   15 +-
 .../hbase/master/snapshot/SnapshotManager.java  |   21 +-
 .../master/snapshot/TakeSnapshotHandler.java    |    2 +-
 .../hadoop/hbase/regionserver/HRegion.java      |    2 +-
 .../hbase/regionserver/HRegionServer.java       |    2 -
 .../snapshot/FlushSnapshotSubprocedure.java     |    3 +-
 .../snapshot/RegionServerSnapshotManager.java   |    2 +-
 .../security/access/AccessControlLists.java     |    2 +-
 .../hbase/security/access/AccessController.java |    2 +-
 .../hadoop/hbase/snapshot/CreateSnapshot.java   |    8 +-
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |    2 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |   30 +-
 .../snapshot/SnapshotDescriptionUtils.java      |   51 +-
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |    2 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |    2 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |    2 +-
 .../hbase/snapshot/SnapshotManifestV2.java      |    4 +-
 .../hbase/snapshot/SnapshotReferenceUtil.java   |    2 +-
 .../resources/hbase-webapps/master/snapshot.jsp |    2 +-
 .../hbase-webapps/master/snapshotsStats.jsp     |    2 +-
 .../hbase/client/TestSnapshotFromClient.java    |    2 +-
 .../hbase/client/TestSnapshotWithAcl.java       |  243 +++
 .../hbase/coprocessor/TestMasterObserver.java   |    2 +-
 .../master/cleaner/TestSnapshotFromMaster.java  |    2 +-
 .../master/snapshot/TestSnapshotFileCache.java  |    1 -
 .../hbase/security/access/SecureTestUtil.java   |    2 +-
 .../security/access/TestAccessController.java   |    2 +-
 .../access/TestWithDisabledAuthorization.java   |    2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    6 +-
 .../hbase/snapshot/TestExportSnapshot.java      |    2 +-
 .../snapshot/TestFlushSnapshotFromClient.java   |    2 +-
 .../TestRestoreFlushSnapshotFromClient.java     |    2 +-
 .../snapshot/TestRestoreSnapshotHelper.java     |    2 +-
 .../snapshot/TestSnapshotClientRetries.java     |    2 +-
 .../snapshot/TestSnapshotDescriptionUtils.java  |    5 +-
 .../hbase/snapshot/TestSnapshotManifest.java    |    3 +-
 hbase-shell/src/main/ruby/hbase.rb              |    1 +
 hbase-shell/src/main/ruby/hbase/admin.rb        |   12 +-
 .../main/ruby/shell/commands/clone_snapshot.rb  |   11 +-
 .../ruby/shell/commands/restore_snapshot.rb     |   10 +-
 69 files changed, 2694 insertions(+), 1994 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/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 82df3f4..5b3744a 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -1146,7 +1147,7 @@ public interface Admin extends Abortable, Closeable {
   @Deprecated
   void snapshot(final String snapshotName,
       final TableName tableName,
-      HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
       IllegalArgumentException;
 
   /**
@@ -1168,7 +1169,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
   @Deprecated
-  void snapshot(HBaseProtos.SnapshotDescription snapshot)
+  void snapshot(SnapshotDescription snapshot)
       throws IOException, SnapshotCreationException, IllegalArgumentException;
 
   /**
@@ -1182,7 +1183,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
   @Deprecated
-  MasterProtos.SnapshotResponse takeSnapshotAsync(HBaseProtos.SnapshotDescription snapshot)
+  MasterProtos.SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot)
       throws IOException, SnapshotCreationException;
 
   /**
@@ -1202,7 +1203,7 @@ public interface Admin extends Abortable, Closeable {
    * unknown
    */
   @Deprecated
-  boolean isSnapshotFinished(final HBaseProtos.SnapshotDescription snapshot)
+  boolean isSnapshotFinished(final SnapshotDescription snapshot)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException;
 
   /**
@@ -1269,6 +1270,23 @@ public interface Admin extends Abortable, Closeable {
       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
+   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
+   * failsafe snapshot name is configurable by using the property
+   * "hbase.snapshot.restore.failsafe.name".
+   * @param snapshotName name of the snapshot to restore
+   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
+   * @param restoreAcl true to restore acl of snapshot into table.
+   * @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
+   */
+  void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
+      throws IOException, RestoreSnapshotException;
+
+  /**
    * Create a new table by cloning the snapshot content.
    *
    * @param snapshotName name of the snapshot to be cloned
@@ -1295,6 +1313,19 @@ public interface Admin extends Abortable, Closeable {
       throws IOException, TableExistsException, RestoreSnapshotException;
 
   /**
+   * Create a new table by cloning the snapshot content.
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @param restoreAcl true to restore acl of snapshot into newly created table
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be created already exists
+   * @throws RestoreSnapshotException if snapshot failed to be cloned
+   * @throws IllegalArgumentException if the specified table has not a valid name
+   */
+  void cloneSnapshot(final String snapshotName, final TableName tableName, final boolean restoreAcl)
+      throws IOException, TableExistsException, RestoreSnapshotException;
+
+  /**
    * Execute a distributed procedure on a cluster.
    *
    * @param signature A distributed procedure is uniquely identified by its signature (default the
@@ -1342,7 +1373,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a network error occurs
    */
   @Deprecated
-  List<HBaseProtos.SnapshotDescription> listSnapshots() throws IOException;
+  List<SnapshotDescription> listSnapshots() throws IOException;
 
   /**
    * List all the completed snapshots matching the given regular expression.
@@ -1352,7 +1383,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   @Deprecated
-  List<HBaseProtos.SnapshotDescription> listSnapshots(String regex) throws IOException;
+  List<SnapshotDescription> listSnapshots(String regex) throws IOException;
 
   /**
    * List all the completed snapshots matching the given pattern.
@@ -1362,7 +1393,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   @Deprecated
-  List<HBaseProtos.SnapshotDescription> listSnapshots(Pattern pattern) throws IOException;
+  List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException;
 
   /**
    * List all the completed snapshots matching the given table name regular expression and snapshot
@@ -1373,7 +1404,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   @Deprecated
-  List<HBaseProtos.SnapshotDescription> listTableSnapshots(String tableNameRegex,
+  List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
       String snapshotNameRegex) throws IOException;
 
   /**
@@ -1385,7 +1416,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   @Deprecated
-  List<HBaseProtos.SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
+  List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/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 6aed027..2cbeb9a 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
@@ -95,7 +95,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 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.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
@@ -157,6 +156,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -917,7 +917,6 @@ public class HBaseAdmin implements Admin {
    * or TimeoutException in case the wait timeout was not long enough to allow the
    * operation to complete.
    *
-   * @param desc table descriptor for table
    * @param tableName name of table to delete
    * @throws IOException if a remote or network exception occurs
    * @return the result of the async delete. You can use Future.get(long, TimeUnit)
@@ -3820,23 +3819,21 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * 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 restored.
-   * If the restore completes without problem the failsafe snapshot is deleted.
-   *
-   * The failsafe snapshot name is configurable by using the property
+   * 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
+   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
+   * failsafe snapshot name is configurable by using the property
    * "hbase.snapshot.restore.failsafe.name".
-   *
    * @param snapshotName name of the snapshot to restore
    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
+   * @param restoreAcl true to restore acl of snapshot into table.
    * @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
    */
   @Override
-  public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
+  public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
       throws IOException, RestoreSnapshotException {
     TableName tableName = null;
     for (SnapshotDescription snapshotInfo: listSnapshots()) {
@@ -3853,7 +3850,7 @@ public class HBaseAdmin implements Admin {
 
     // The table does not exists, switch to clone.
     if (!tableExists(tableName)) {
-      cloneSnapshot(snapshotName, tableName);
+      cloneSnapshot(snapshotName, tableName, restoreAcl);
       return;
     }
 
@@ -3877,13 +3874,13 @@ public class HBaseAdmin implements Admin {
 
     try {
       // Restore snapshot
-      internalRestoreSnapshot(snapshotName, tableName);
+      internalRestoreSnapshot(snapshotName, tableName, restoreAcl);
     } 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);
+          internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl);
           String msg = "Restore snapshot=" + snapshotName +
             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
           LOG.error(msg, e);
@@ -3909,6 +3906,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @Override
+  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
+      throws IOException, RestoreSnapshotException {
+    restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
+  }
+
   /**
    * Create a new table by cloning the snapshot content.
    *
@@ -3968,15 +3971,21 @@ public class HBaseAdmin implements Admin {
    * @throws IllegalArgumentException if the specified table has not a valid name
    */
   @Override
-  public void cloneSnapshot(final String snapshotName, final TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
+  public void cloneSnapshot(final String snapshotName, final TableName tableName,
+      final boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
     if (tableExists(tableName)) {
       throw new TableExistsException(tableName);
     }
-    internalRestoreSnapshot(snapshotName, tableName);
+    internalRestoreSnapshot(snapshotName, tableName, restoreAcl);
     waitUntilTableIsEnabled(tableName);
   }
 
+  @Override
+  public void cloneSnapshot(String snapshotName, TableName tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    cloneSnapshot(snapshotName, tableName, false);
+  }
+
   /**
    * Execute a distributed procedure on a cluster synchronously with return data
    *
@@ -4117,23 +4126,23 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
-   * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
-   * create an HTable instance to this table before it is available.
+   * Execute Restore/Clone snapshot and wait for the server to complete (blocking). To check if the
+   * cloned table exists, use {@link #isTableAvailable} -- it is not safe to create an HTable
+   * instance to this table before it is available.
    * @param snapshotName snapshot to restore
    * @param tableName table name to restore the snapshot on
    * @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 void internalRestoreSnapshot(final String snapshotName, final TableName
-      tableName)
+  private void internalRestoreSnapshot(final String snapshotName, final TableName tableName,
+      final boolean restoreAcl)
       throws IOException, RestoreSnapshotException {
     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
 
     // actually restore the snapshot
-    internalRestoreSnapshotAsync(snapshot);
+    internalRestoreSnapshotAsync(snapshot, restoreAcl);
 
     final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
         .setSnapshot(snapshot).build();
@@ -4177,12 +4186,12 @@ public class HBaseAdmin implements Admin {
    * @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 {
+  private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot,
+      final boolean restoreAcl) throws IOException, RestoreSnapshotException {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
 
-    final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
-        .build();
+    final RestoreSnapshotRequest request =
+        RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setRestoreACL(restoreAcl).build();
 
     // run the snapshot restore on the master
     return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
index 4fb7548..499a9bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
@@ -157,6 +157,10 @@ public class TablePermission extends Permission {
     return table;
   }
 
+  public void setTableName(TableName table) {
+    this.table = table;
+  }
+
   public boolean hasFamily() {
     return family != null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
index d439c8b..686dbb8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -36,7 +36,7 @@ public class ClientSnapshotDescriptionUtils {
    * @throws IllegalArgumentException if the name of the snapshot or the name of the table to
    *           snapshot are not valid names.
    */
-  public static void assertSnapshotRequestIsValid(HBaseProtos.SnapshotDescription snapshot)
+  public static void assertSnapshotRequestIsValid(SnapshotDescription snapshot)
       throws IllegalArgumentException {
     // make sure the snapshot name is valid
     TableName.isLegalTableQualifierName(Bytes.toBytes(snapshot.getName()), true);
@@ -52,12 +52,12 @@ public class ClientSnapshotDescriptionUtils {
 
   /**
    * Returns a single line (no \n) representation of snapshot metadata.  Use this instead of
-   * {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription#toString()}.  We don't replace SnapshotDescrpition's toString
+   * {@link SnapshotDescription#toString()}.  We don't replace SnapshotDescrpition's toString
    * because it is auto-generated by protoc.
    * @param ssd
    * @return Single line string with a summary of the snapshot parameters
    */
-  public static String toString(HBaseProtos.SnapshotDescription ssd) {
+  public static String toString(SnapshotDescription ssd) {
     if (ssd == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
index d29c89c..a1f5911 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
index cd2f66f..a88d820 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
  * General exception base class for when a snapshot fails

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
index 25cd583..8ed42e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot could not be restored due to a server-side error when restoring it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
index 324d41f..d550101 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot could not be created due to a server-side error when

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
index 6ba45bd..6006136 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
index ff3cdcb..0344aff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot exists but should not

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
index 6385c27..afa5892 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
@@ -32,11 +32,11 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 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.SnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
+import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9dc4cad/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index 0e91490..3c4fb61 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -11141,1281 +11141,6 @@ public final class HBaseProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.NameInt64Pair)
   }
 
-  public interface SnapshotDescriptionOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required string name = 1;
-    /**
-     * <code>required string name = 1;</code>
-     */
-    boolean hasName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    java.lang.String getName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    com.google.protobuf.ByteString
-        getNameBytes();
-
-    // optional string table = 2;
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    boolean hasTable();
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    java.lang.String getTable();
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    com.google.protobuf.ByteString
-        getTableBytes();
-
-    // optional int64 creation_time = 3 [default = 0];
-    /**
-     * <code>optional int64 creation_time = 3 [default = 0];</code>
-     */
-    boolean hasCreationTime();
-    /**
-     * <code>optional int64 creation_time = 3 [default = 0];</code>
-     */
-    long getCreationTime();
-
-    // optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];
-    /**
-     * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-     */
-    boolean hasType();
-    /**
-     * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType();
-
-    // optional int32 version = 5;
-    /**
-     * <code>optional int32 version = 5;</code>
-     */
-    boolean hasVersion();
-    /**
-     * <code>optional int32 version = 5;</code>
-     */
-    int getVersion();
-
-    // optional string owner = 6;
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    boolean hasOwner();
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    java.lang.String getOwner();
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    com.google.protobuf.ByteString
-        getOwnerBytes();
-  }
-  /**
-   * Protobuf type {@code hbase.pb.SnapshotDescription}
-   *
-   * <pre>
-   **
-   * Description of the snapshot to take
-   * </pre>
-   */
-  public static final class SnapshotDescription extends
-      com.google.protobuf.GeneratedMessage
-      implements SnapshotDescriptionOrBuilder {
-    // Use SnapshotDescription.newBuilder() to construct.
-    private SnapshotDescription(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private SnapshotDescription(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final SnapshotDescription defaultInstance;
-    public static SnapshotDescription getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public SnapshotDescription getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private SnapshotDescription(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              name_ = input.readBytes();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              table_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              creationTime_ = input.readInt64();
-              break;
-            }
-            case 32: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(4, rawValue);
-              } else {
-                bitField0_ |= 0x00000008;
-                type_ = value;
-              }
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000010;
-              version_ = input.readInt32();
-              break;
-            }
-            case 50: {
-              bitField0_ |= 0x00000020;
-              owner_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<SnapshotDescription> PARSER =
-        new com.google.protobuf.AbstractParser<SnapshotDescription>() {
-      public SnapshotDescription parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SnapshotDescription(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<SnapshotDescription> getParserForType() {
-      return PARSER;
-    }
-
-    /**
-     * Protobuf enum {@code hbase.pb.SnapshotDescription.Type}
-     */
-    public enum Type
-        implements com.google.protobuf.ProtocolMessageEnum {
-      /**
-       * <code>DISABLED = 0;</code>
-       */
-      DISABLED(0, 0),
-      /**
-       * <code>FLUSH = 1;</code>
-       */
-      FLUSH(1, 1),
-      /**
-       * <code>SKIPFLUSH = 2;</code>
-       */
-      SKIPFLUSH(2, 2),
-      ;
-
-      /**
-       * <code>DISABLED = 0;</code>
-       */
-      public static final int DISABLED_VALUE = 0;
-      /**
-       * <code>FLUSH = 1;</code>
-       */
-      public static final int FLUSH_VALUE = 1;
-      /**
-       * <code>SKIPFLUSH = 2;</code>
-       */
-      public static final int SKIPFLUSH_VALUE = 2;
-
-
-      public final int getNumber() { return value; }
-
-      public static Type valueOf(int value) {
-        switch (value) {
-          case 0: return DISABLED;
-          case 1: return FLUSH;
-          case 2: return SKIPFLUSH;
-          default: return null;
-        }
-      }
-
-      public static com.google.protobuf.Internal.EnumLiteMap<Type>
-          internalGetValueMap() {
-        return internalValueMap;
-      }
-      private static com.google.protobuf.Internal.EnumLiteMap<Type>
-          internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap<Type>() {
-              public Type findValueByNumber(int number) {
-                return Type.valueOf(number);
-              }
-            };
-
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
-          getValueDescriptor() {
-        return getDescriptor().getValues().get(index);
-      }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptorForType() {
-        return getDescriptor();
-      }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDescriptor().getEnumTypes().get(0);
-      }
-
-      private static final Type[] VALUES = values();
-
-      public static Type valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-        if (desc.getType() != getDescriptor()) {
-          throw new java.lang.IllegalArgumentException(
-            "EnumValueDescriptor is not for this type.");
-        }
-        return VALUES[desc.getIndex()];
-      }
-
-      private final int index;
-      private final int value;
-
-      private Type(int index, int value) {
-        this.index = index;
-        this.value = value;
-      }
-
-      // @@protoc_insertion_point(enum_scope:hbase.pb.SnapshotDescription.Type)
-    }
-
-    private int bitField0_;
-    // required string name = 1;
-    public static final int NAME_FIELD_NUMBER = 1;
-    private java.lang.Object name_;
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public boolean hasName() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public java.lang.String getName() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          name_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public com.google.protobuf.ByteString
-        getNameBytes() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        name_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional string table = 2;
-    public static final int TABLE_FIELD_NUMBER = 2;
-    private java.lang.Object table_;
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    public boolean hasTable() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    public java.lang.String getTable() {
-      java.lang.Object ref = table_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          table_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string table = 2;</code>
-     *
-     * <pre>
-     * not needed for delete, but checked for in taking snapshot
-     * </pre>
-     */
-    public com.google.protobuf.ByteString
-        getTableBytes() {
-      java.lang.Object ref = table_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        table_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional int64 creation_time = 3 [default = 0];
-    public static final int CREATION_TIME_FIELD_NUMBER = 3;
-    private long creationTime_;
-    /**
-     * <code>optional int64 creation_time = 3 [default = 0];</code>
-     */
-    public boolean hasCreationTime() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 creation_time = 3 [default = 0];</code>
-     */
-    public long getCreationTime() {
-      return creationTime_;
-    }
-
-    // optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];
-    public static final int TYPE_FIELD_NUMBER = 4;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type_;
-    /**
-     * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-     */
-    public boolean hasType() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType() {
-      return type_;
-    }
-
-    // optional int32 version = 5;
-    public static final int VERSION_FIELD_NUMBER = 5;
-    private int version_;
-    /**
-     * <code>optional int32 version = 5;</code>
-     */
-    public boolean hasVersion() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional int32 version = 5;</code>
-     */
-    public int getVersion() {
-      return version_;
-    }
-
-    // optional string owner = 6;
-    public static final int OWNER_FIELD_NUMBER = 6;
-    private java.lang.Object owner_;
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    public boolean hasOwner() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    public java.lang.String getOwner() {
-      java.lang.Object ref = owner_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          owner_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string owner = 6;</code>
-     */
-    public com.google.protobuf.ByteString
-        getOwnerBytes() {
-      java.lang.Object ref = owner_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        owner_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    private void initFields() {
-      name_ = "";
-      table_ = "";
-      creationTime_ = 0L;
-      type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
-      version_ = 0;
-      owner_ = "";
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasName()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getTableBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(3, creationTime_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeEnum(4, type_.getNumber());
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeInt32(5, version_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeBytes(6, getOwnerBytes());
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getTableBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(3, creationTime_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(4, type_.getNumber());
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(5, version_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(6, getOwnerBytes());
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription) obj;
-
-      boolean result = true;
-      result = result && (hasName() == other.hasName());
-      if (hasName()) {
-        result = result && getName()
-            .equals(other.getName());
-      }
-      result = result && (hasTable() == other.hasTable());
-      if (hasTable()) {
-        result = result && getTable()
-            .equals(other.getTable());
-      }
-      result = result && (hasCreationTime() == other.hasCreationTime());
-      if (hasCreationTime()) {
-        result = result && (getCreationTime()
-            == other.getCreationTime());
-      }
-      result = result && (hasType() == other.hasType());
-      if (hasType()) {
-        result = result &&
-            (getType() == other.getType());
-      }
-      result = result && (hasVersion() == other.hasVersion());
-      if (hasVersion()) {
-        result = result && (getVersion()
-            == other.getVersion());
-      }
-      result = result && (hasOwner() == other.hasOwner());
-      if (hasOwner()) {
-        result = result && getOwner()
-            .equals(other.getOwner());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasName()) {
-        hash = (37 * hash) + NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getName().hashCode();
-      }
-      if (hasTable()) {
-        hash = (37 * hash) + TABLE_FIELD_NUMBER;
-        hash = (53 * hash) + getTable().hashCode();
-      }
-      if (hasCreationTime()) {
-        hash = (37 * hash) + CREATION_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getCreationTime());
-      }
-      if (hasType()) {
-        hash = (37 * hash) + TYPE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getType());
-      }
-      if (hasVersion()) {
-        hash = (37 * hash) + VERSION_FIELD_NUMBER;
-        hash = (53 * hash) + getVersion();
-      }
-      if (hasOwner()) {
-        hash = (37 * hash) + OWNER_FIELD_NUMBER;
-        hash = (53 * hash) + getOwner().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.SnapshotDescription}
-     *
-     * <pre>
-     **
-     * Description of the snapshot to take
-     * </pre>
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        name_ = "";
-        bitField0_ = (bitField0_ & ~0x00000001);
-        table_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        creationTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        version_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        owner_ = "";
-        bitField0_ = (bitField0_ & ~0x00000020);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription build() {
-        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.name_ = name_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.table_ = table_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.creationTime_ = creationTime_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.type_ = type_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.version_ = version_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.owner_ = owner_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.getDefaultInstance()) return this;
-        if (other.hasName()) {
-          bitField0_ |= 0x00000001;
-          name_ = other.name_;
-          onChanged();
-        }
-        if (other.hasTable()) {
-          bitField0_ |= 0x00000002;
-          table_ = other.table_;
-          onChanged();
-        }
-        if (other.hasCreationTime()) {
-          setCreationTime(other.getCreationTime());
-        }
-        if (other.hasType()) {
-          setType(other.getType());
-        }
-        if (other.hasVersion()) {
-          setVersion(other.getVersion());
-        }
-        if (other.hasOwner()) {
-          bitField0_ |= 0x00000020;
-          owner_ = other.owner_;
-          onChanged();
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasName()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required string name = 1;
-      private java.lang.Object name_ = "";
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public boolean hasName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public java.lang.String getName() {
-        java.lang.Object ref = name_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          name_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public com.google.protobuf.ByteString
-          getNameBytes() {
-        java.lang.Object ref = name_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          name_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder setName(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        name_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder clearName() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        name_ = getDefaultInstance().getName();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        name_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional string table = 2;
-      private java.lang.Object table_ = "";
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public boolean hasTable() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public java.lang.String getTable() {
-        java.lang.Object ref = table_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          table_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public com.google.protobuf.ByteString
-          getTableBytes() {
-        java.lang.Object ref = table_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          table_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public Builder setTable(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        table_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public Builder clearTable() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        table_ = getDefaultInstance().getTable();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string table = 2;</code>
-       *
-       * <pre>
-       * not needed for delete, but checked for in taking snapshot
-       * </pre>
-       */
-      public Builder setTableBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        table_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 creation_time = 3 [default = 0];
-      private long creationTime_ ;
-      /**
-       * <code>optional int64 creation_time = 3 [default = 0];</code>
-       */
-      public boolean hasCreationTime() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional int64 creation_time = 3 [default = 0];</code>
-       */
-      public long getCreationTime() {
-        return creationTime_;
-      }
-      /**
-       * <code>optional int64 creation_time = 3 [default = 0];</code>
-       */
-      public Builder setCreationTime(long value) {
-        bitField0_ |= 0x00000004;
-        creationTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 creation_time = 3 [default = 0];</code>
-       */
-      public Builder clearCreationTime() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        creationTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
-      /**
-       * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-       */
-      public boolean hasType() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type getType() {
-        return type_;
-      }
-      /**
-       * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-       */
-      public Builder setType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        bitField0_ |= 0x00000008;
-        type_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.SnapshotDescription.Type type = 4 [default = FLUSH];</code>
-       */
-      public Builder clearType() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        type_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type.FLUSH;
-        onChanged();
-        return this;
-      }
-
-      // optional int32 version = 5;
-      private int version_ ;
-      /**
-       * <code>optional int32 version = 5;</code>
-       */
-      public boolean hasVersion() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional int32 version = 5;</code>
-       */
-      public int getVersion() {
-        return version_;
-      }
-      /**
-       * <code>optional int32 version = 5;</code>
-       */
-      public Builder setVersion(int value) {
-        bitField0_ |= 0x00000010;
-        version_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 version = 5;</code>
-       */
-      public Builder clearVersion() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        version_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional string owner = 6;
-      private java.lang.Object owner_ = "";
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public boolean hasOwner() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public java.lang.String getOwner() {
-        java.lang.Object ref = owner_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          owner_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public com.google.protobuf.ByteString
-          getOwnerBytes() {
-        java.lang.Object ref = owner_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          owner_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public Builder setOwner(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        owner_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public Builder clearOwner() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        owner_ = getDefaultInstance().getOwner();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string owner = 6;</code>
-       */
-      public Builder setOwnerBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        owner_ = value;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotDescription)
-    }
-
-    static {
-      defaultInstance = new SnapshotDescription(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotDescription)
-  }
-
   public interface ProcedureDescriptionOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -19004,11 +17729,6 @@ public final class HBaseProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotDescription_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureDescription_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -19092,36 +17812,30 @@ public final class HBaseProtos {
       "\002(\t\",\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005va" +
       "lue\030\002 \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002" +
       "(\014\022\016\n\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004n",
-      "ame\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\325\001\n\023SnapshotDes" +
-      "cription\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n" +
-      "\rcreation_time\030\003 \001(\003:\0010\0227\n\004type\030\004 \001(\0162\"." +
-      "hbase.pb.SnapshotDescription.Type:\005FLUSH" +
-      "\022\017\n\007version\030\005 \001(\005\022\r\n\005owner\030\006 \001(\t\".\n\004Type" +
-      "\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002" +
-      "\"\206\001\n\024ProcedureDescription\022\021\n\tsignature\030\001" +
-      " \002(\t\022\020\n\010instance\030\002 \001(\t\022\030\n\rcreation_time\030" +
-      "\003 \001(\003:\0010\022/\n\rconfiguration\030\004 \003(\0132\030.hbase." +
-      "pb.NameStringPair\"\n\n\010EmptyMsg\"\033\n\007LongMsg",
-      "\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDoubleMsg\022\022\n\ndoubl" +
-      "e_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg\022\026\n\016bigdecim" +
-      "al_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016least_sig_bits\030\001" +
-      " \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"T\n\023Namespace" +
-      "Descriptor\022\014\n\004name\030\001 \002(\014\022/\n\rconfiguratio" +
-      "n\030\002 \003(\0132\030.hbase.pb.NameStringPair\"\235\001\n\013Ve" +
-      "rsionInfo\022\017\n\007version\030\001 \002(\t\022\013\n\003url\030\002 \002(\t\022" +
-      "\020\n\010revision\030\003 \002(\t\022\014\n\004user\030\004 \002(\t\022\014\n\004date\030" +
-      "\005 \002(\t\022\024\n\014src_checksum\030\006 \002(\t\022\025\n\rversion_m" +
-      "ajor\030\007 \001(\r\022\025\n\rversion_minor\030\010 \001(\r\"Q\n\020Reg",
-      "ionServerInfo\022\020\n\010infoPort\030\001 \001(\005\022+\n\014versi" +
-      "on_info\030\002 \001(\0132\025.hbase.pb.VersionInfo*r\n\013" +
-      "CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001" +
-      "\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR" +
-      "_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010Time" +
-      "Unit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022" +
-      "\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTE" +
-      "S\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.h" +
-      "adoop.hbase.protobuf.generatedB\013HBasePro" +
-      "tosH\001\240\001\001"
+      "ame\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\206\001\n\024ProcedureDe" +
+      "scription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instance" +
+      "\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022/\n\rconf" +
+      "iguration\030\004 \003(\0132\030.hbase.pb.NameStringPai" +
+      "r\"\n\n\010EmptyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002" +
+      "(\003\"\037\n\tDoubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rB" +
+      "igDecimalMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004" +
+      "UUID\022\026\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig" +
+      "_bits\030\002 \002(\004\"T\n\023NamespaceDescriptor\022\014\n\004na" +
+      "me\030\001 \002(\014\022/\n\rconfiguration\030\002 \003(\0132\030.hbase.",
+      "pb.NameStringPair\"\235\001\n\013VersionInfo\022\017\n\007ver" +
+      "sion\030\001 \002(\t\022\013\n\003url\030\002 \002(\t\022\020\n\010revision\030\003 \002(" +
+      "\t\022\014\n\004user\030\004 \002(\t\022\014\n\004date\030\005 \002(\t\022\024\n\014src_che" +
+      "cksum\030\006 \002(\t\022\025\n\rversion_major\030\007 \001(\r\022\025\n\rve" +
+      "rsion_minor\030\010 \001(\r\"Q\n\020RegionServerInfo\022\020\n" +
+      "\010infoPort\030\001 \001(\005\022+\n\014version_info\030\002 \001(\0132\025." +
+      "hbase.pb.VersionInfo*r\n\013CompareType\022\010\n\004L" +
+      "ESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tN" +
+      "OT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREA" +
+      "TER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSECO",
+      "NDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECONDS\020" +
+      "\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006\022\010" +
+      "\n\004DAYS\020\007B>\n*org.apache.hadoop.hbase.prot" +
+      "obuf.generatedB\013HBaseProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -19212,62 +17926,56 @@ public final class HBaseProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_NameInt64Pair_descriptor,
               new java.lang.String[] { "Name", "Value", });
-          internal_static_hbase_pb_SnapshotDescription_descriptor =
-            getDescriptor().getMessageTypes().get(14);
-          internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SnapshotDescription_descriptor,
-              new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", "Owner", });
           internal_static_hbase_pb_ProcedureDescription_descriptor =
-            getDescriptor().getMessageTypes().get(15);
+            getDescriptor().getMessageTypes().get(14);
           internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ProcedureDescription_descriptor,
               new java.lang.String[] { "Signature", "Instance", "CreationTime", "Configuration", });
           internal_static_hbase_pb_EmptyMsg_descriptor =
-            getDescriptor().getMessageTypes().get(16);
+            getDescriptor().getMessageTypes().get(15);
           internal_static_hbase_pb_EmptyMsg_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_EmptyMsg_descriptor,
               new java.lang.String[] { });
           internal_static_hbase_pb_LongMsg_descriptor =
-            getDescriptor().getMessageTypes().get(17);
+            getDescriptor().getMessageTypes().get(16);
           internal_static_hbase_pb_LongMsg_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_LongMsg_descriptor,
               new java.lang.String[] { "LongMsg", });
           internal_static_hbase_pb_DoubleMsg_descriptor =
-            getDescriptor().getMessageTypes().get(18);
+            getDescriptor().getMessageTypes().get(17);
           internal_static_hbase_pb_DoubleMsg_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_DoubleMsg_descriptor,
               new java.lang.String[] { "DoubleMsg", });
           internal_static_hbase_pb_BigDecimalMsg_descriptor =
-            getDescriptor().getMessageTypes().get(19);
+            getDescriptor().getMessageTypes().get(18);
           internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_BigDecimalMsg_descriptor,
               new java.lang.String[] { "BigdecimalMsg", });
           internal_static_hbase_pb_UUID_descriptor =
-            getDescriptor().getMessageTypes().get(20);
+            getDescriptor().getMessageTypes().get(19);
           internal_static_hbase_pb_UUID_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_UUID_descriptor,
               new java.lang.String[] { "LeastSigBits", "MostSigBits", });
           internal_static_hbase_pb_NamespaceDescriptor_descriptor =
-            getDescriptor().getMessageTypes().get(21);
+            getDescriptor().getMessageTypes().get(20);
           internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_NamespaceDescriptor_descriptor,
               new java.lang.String[] { "Name", "Configuration", });
           internal_static_hbase_pb_VersionInfo_descriptor =
-            getDescriptor().getMessageTypes().get(22);
+            getDescriptor().getMessageTypes().get(21);
           internal_static_hbase_pb_VersionInfo_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_VersionInfo_descriptor,
               new java.lang.String[] { "Version", "Url", "Revision", "User", "Date", "SrcChecksum", "VersionMajor", "VersionMinor", });
           internal_static_hbase_pb_RegionServerInfo_descriptor =
-            getDescriptor().getMessageTypes().get(23);
+            getDescriptor().getMessageTypes().get(22);
           internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionServerInfo_descriptor,