hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject [6/6] hbase git commit: HBASE-15411 Rewrite backup with Procedure V2 - phase 1
Date Fri, 01 Apr 2016 22:02:28 GMT
HBASE-15411 Rewrite backup with Procedure V2 - phase 1


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

Branch: refs/heads/HBASE-7912
Commit: b37cc7606f498eef7190998a528be2e18d206a06
Parents: 1e9470d
Author: tedyu <yuzhihong@gmail.com>
Authored: Fri Apr 1 15:02:22 2016 -0700
Committer: tedyu <yuzhihong@gmail.com>
Committed: Fri Apr 1 15:02:22 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/BackupRequest.java      |   82 +
 .../org/apache/hadoop/hbase/client/Admin.java   |   31 +
 .../hbase/client/ConnectionImplementation.java  |    7 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  121 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |    6 +
 .../hadoop/hbase/protobuf/RequestConverter.java |   18 +
 .../hbase/protobuf/generated/BackupProtos.java  | 6099 ++++++++++++------
 .../hbase/protobuf/generated/MasterProtos.java  | 2567 +++++++-
 hbase-protocol/src/main/protobuf/Backup.proto   |   24 +
 hbase-protocol/src/main/protobuf/Master.proto   |   20 +-
 .../hadoop/hbase/backup/BackupClient.java       |   40 -
 .../hbase/backup/BackupRestoreFactory.java      |   16 -
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  340 +-
 .../hbase/backup/impl/BackupClientImpl.java     |  183 -
 .../hbase/backup/impl/BackupCommands.java       |   21 +-
 .../hadoop/hbase/backup/impl/BackupContext.java |   46 +-
 .../hadoop/hbase/backup/impl/BackupHandler.java |  702 --
 .../hadoop/hbase/backup/impl/BackupManager.java |   37 +-
 .../hbase/backup/impl/BackupSnapshotCopy.java   |    8 +-
 .../hbase/backup/impl/BackupSystemTable.java    |   31 +-
 .../backup/impl/BackupSystemTableHelper.java    |   34 +-
 .../hadoop/hbase/backup/impl/BackupUtil.java    |   49 +-
 .../backup/impl/FullTableBackupProcedure.java   |  745 +++
 .../impl/IncrementalTableBackupProcedure.java   |  325 +
 .../hbase/backup/impl/RestoreClientImpl.java    |    3 +-
 .../hadoop/hbase/backup/impl/RestoreUtil.java   |  347 +-
 .../mapreduce/MapReduceBackupCopyService.java   |   25 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   74 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   20 +
 .../hadoop/hbase/master/MasterServices.java     |   20 +
 .../master/procedure/MasterProcedureUtil.java   |   69 +
 .../procedure/TableProcedureInterface.java      |    2 +-
 .../hadoop/hbase/backup/TestBackupBase.java     |   36 +-
 .../hbase/backup/TestBackupBoundaryTests.java   |   16 +-
 .../hbase/backup/TestBackupLogCleaner.java      |   10 +-
 .../hbase/backup/TestBackupSystemTable.java     |    9 +-
 .../hadoop/hbase/backup/TestFullBackup.java     |   13 +-
 .../hadoop/hbase/backup/TestFullRestore.java    |   32 +-
 .../hbase/backup/TestIncrementalBackup.java     |   34 +-
 .../hadoop/hbase/backup/TestRemoteBackup.java   |   10 +-
 .../hadoop/hbase/backup/TestRemoteRestore.java  |    7 +-
 .../hbase/backup/TestRestoreBoundaryTests.java  |   12 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   12 +
 43 files changed, 8341 insertions(+), 3962 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..bbf421c
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO class for backup request
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupRequest {
+  private BackupType type;
+  private List<TableName> tableList;
+  private String targetRootDir;
+  private int workers = -1;
+  private long bandwidth = -1L;
+
+  public BackupRequest() {
+  }
+
+  public BackupRequest setBackupType(BackupType type) {
+    this.type = type;
+    return this;
+  }
+  public BackupType getBackupType() {
+    return this.type;
+  }
+
+  public BackupRequest setTableList(List<TableName> tableList) {
+    this.tableList = tableList;
+    return this;
+  }
+  public List<TableName> getTableList() {
+    return this.tableList;
+  }
+
+  public BackupRequest setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+    return this;
+  }
+  public String getTargetRootDir() {
+    return this.targetRootDir;
+  }
+
+  public BackupRequest setWorkers(int workers) {
+    this.workers = workers;
+    return this;
+  }
+  public int getWorkers() {
+    return this.workers;
+  }
+
+  public BackupRequest setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+    return this;
+  }
+  public long getBandwidth() {
+    return this.bandwidth;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/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..02ffc18 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
@@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -913,6 +915,35 @@ public interface Admin extends Abortable, Closeable {
     throws IOException;
 
   /**
+   * Backs up given list of tables fully. Asynchronous operation. This means that
+   * it may be a while before all your tables are backed up.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * 
+   * @param request BackupRequest instance which contains the following members:
+   *  type whether the backup is full or incremental
+   *  tableList list of tables to backup
+   *  targetRootDir root dir for saving the backup
+   *  workers number of paralle workers. -1 - system defined
+   *  bandwidth bandwidth per worker in MB per sec. -1 - unlimited
+   * @return the result of the async backup. You can use Future.get(long, TimeUnit) to wait
on the
+   *     operation to complete
+   */
+  public Future<String> backupTablesAsync(final BackupRequest request) throws IOException;
+
+  /**
+   * Backs up given list of tables fully. Synchronous operation.
+   * 
+   * @param request BackupRequest instance which contains the following members:
+   *  type whether the backup is full or incremental
+   *  tableList list of tables to backup
+   *  targetRootDir root dir for saving the backup
+   *  workers number of paralle workers. -1 - system defined
+   *  bandwidth bandwidth per worker in MB per sec. -1 - unlimited
+   * @return the backup Id
+   */
+  public String backupTables(final BackupRequest request) throws IOException;
+
+  /**
    * Modify an existing table, more IRB friendly version. Asynchronous operation.  This means
that
    * it may be a while before your schema change is updated across all of the table.
    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/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 64eb9fb..c245ef4 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
@@ -1401,6 +1401,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable
{
       }
 
       @Override
+      public MasterProtos.BackupTablesResponse backupTables(
+          RpcController controller,
+          MasterProtos.BackupTablesRequest request)  throws ServiceException {
+        return stub.backupTables(controller, request);
+      }
+
+      @Override
       public MasterProtos.AddColumnResponse addColumn(
           RpcController controller,
           MasterProtos.AddColumnRequest request) throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/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 c2a0bb8..f19da53 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
@@ -61,6 +61,8 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -83,6 +85,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterReque
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesResponse;
 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;
@@ -418,14 +422,6 @@ public class HBaseAdmin implements Admin {
       throw new TableNotFoundException(tableName.getNameAsString());
   }
 
-  private long getPauseTime(int tries) {
-    int triesCount = tries;
-    if (triesCount >= HConstants.RETRY_BACKOFF.length) {
-      triesCount = HConstants.RETRY_BACKOFF.length - 1;
-    }
-    return this.pause * HConstants.RETRY_BACKOFF[triesCount];
-  }
-
   @Override
   public void createTable(HTableDescriptor desc)
   throws IOException {
@@ -690,7 +686,7 @@ public class HBaseAdmin implements Admin {
       if (enabled) {
         break;
       }
-      long sleep = getPauseTime(tries);
+      long sleep = getPauseTime(tries, pause);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
           "enabled in " + tableName);
@@ -1568,6 +1564,73 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException
{
+    BackupClientUtil.checkTargetDir(userRequest.getTargetRootDir(), conf);
+    if (userRequest.getTableList() != null) {
+      for (TableName table : userRequest.getTableList()) {
+        if (!tableExists(table)) {
+          throw new DoNotRetryIOException(table + "does not exist");
+        }
+      }
+    }
+
+    BackupTablesResponse response = executeCallable(
+      new MasterCallable<BackupTablesResponse>(getConnection()) {
+        @Override
+        public BackupTablesResponse call(int callTimeout) throws ServiceException {
+          BackupTablesRequest request = RequestConverter.buildBackupTablesRequest(
+            userRequest.getBackupType(), userRequest.getTableList(), userRequest.getTargetRootDir(),
+            userRequest.getWorkers(), userRequest.getBandwidth());
+          return master.backupTables(null, request);
+        }
+      });
+    return new TableBackupFuture(this, TableName.BACKUP_TABLE_NAME, response);
+  }
+
+  @Override
+  public String backupTables(final BackupRequest userRequest) throws IOException {
+    return get(
+      backupTablesAsync(userRequest),
+      syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  public static class TableBackupFuture extends TableFuture<String> {
+    String backupId;
+    public TableBackupFuture(final HBaseAdmin admin, final TableName tableName,
+        final BackupTablesResponse response) {
+      super(admin, tableName,
+          (response != null && response.hasProcId()) ? response.getProcId() : null);
+      backupId = response.getBackupId();
+    }
+
+    String getBackupId() {
+      return backupId;
+    }
+
+    @Override
+    public String getOperationType() {
+      return "BACKUP";
+    }
+
+    @Override
+    protected String convertResult(final GetProcedureResultResponse response) throws IOException
{
+      if (response.hasException()) {
+        throw ForeignExceptionUtil.toIOException(response.getException());
+      }
+      ByteString result = response.getResult();
+      if (result == null) return null;
+      return Bytes.toStringBinary(result.toByteArray());
+    }
+
+    @Override
+    protected String postOperationResult(final String result,
+      final long deadlineTs) throws IOException, TimeoutException {
+      return result;
+    }
+  }
+
+  @Override
   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor
htd)
   throws IOException {
     if (!tableName.equals(htd.getTableName())) {
@@ -2173,17 +2236,13 @@ public class HBaseAdmin implements Admin {
     snapshot(builder.build());
   }
 
-  @Override
-  public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
-      IllegalArgumentException {
-    // actually take the snapshot
-    SnapshotResponse response = takeSnapshotAsync(snapshot);
+  public void waitForSnapshot(SnapshotDescription snapshot, long max,
+      HConnection conn) throws IOException {
     final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
         .build();
     IsSnapshotDoneResponse done = null;
     long start = EnvironmentEdgeManager.currentTime();
-    long max = response.getExpectedTimeout();
-    long maxPauseTime = max / this.numRetries;
+    long maxPauseTime = max / numRetries;
     int tries = 0;
     LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
         ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
@@ -2192,7 +2251,7 @@ public class HBaseAdmin implements Admin {
         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone()))
{
       try {
         // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
+        long sleep = getPauseTime(tries++, pause);
         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
           "ms while waiting for snapshot completion.");
@@ -2201,7 +2260,7 @@ public class HBaseAdmin implements Admin {
         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
       }
       LOG.debug("Getting current status of snapshot from master...");
-      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection())
{
+      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(conn) {
         @Override
         public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
           return master.isSnapshotDone(null, request);
@@ -2215,6 +2274,14 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
+      IllegalArgumentException {
+    // actually take the snapshot
+    SnapshotResponse response = takeSnapshotAsync(snapshot);
+    waitForSnapshot(snapshot, response.getExpectedTimeout(), getConnection());
+  }
+
+  @Override
   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
       SnapshotCreationException {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
@@ -2377,6 +2444,14 @@ public class HBaseAdmin implements Admin {
     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
   }
 
+  public static long getPauseTime(int tries, long pause) {
+    int triesCount = tries;
+    if (triesCount >= HConstants.RETRY_BACKOFF.length) {
+      triesCount = HConstants.RETRY_BACKOFF.length - 1;
+    }
+    return pause * HConstants.RETRY_BACKOFF[triesCount];
+  }
+
   @Override
   public void execProcedure(String signature, String instance, Map<String, String>
props)
       throws IOException {
@@ -2410,7 +2485,7 @@ public class HBaseAdmin implements Admin {
         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) {
       try {
         // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
+        long sleep = getPauseTime(tries++, pause);
         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
           "ms while waiting for procedure completion.");
@@ -2475,7 +2550,7 @@ public class HBaseAdmin implements Admin {
     while (!done.getDone()) {
       try {
         // sleep a backoff <= pauseTime amount
-        long sleep = getPauseTime(tries++);
+        long sleep = getPauseTime(tries++, pause);
         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
         LOG.debug(tries + ") Sleeping: " + sleep
             + " ms while we wait for snapshot restore to complete.");
@@ -3042,7 +3117,7 @@ public class HBaseAdmin implements Admin {
         }
 
         try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
+          Thread.sleep(getPauseTime(tries++, getAdmin().pause));
         } catch (InterruptedException e) {
           throw new InterruptedException(
             "Interrupted while waiting for the result of proc " + procId);
@@ -3144,7 +3219,7 @@ public class HBaseAdmin implements Admin {
           serverEx = e;
         }
         try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
+          Thread.sleep(getPauseTime(tries++, getAdmin().pause));
         } catch (InterruptedException e) {
           callable.throwInterruptedException();
         }
@@ -3321,7 +3396,7 @@ public class HBaseAdmin implements Admin {
         }
 
         try {
-          Thread.sleep(getAdmin().getPauseTime(tries++));
+          Thread.sleep(getPauseTime(tries++, getAdmin().pause));
         } catch (InterruptedException e) {
           throw new InterruptedIOException("Interrupted when opening" + " regions; "
               + actualRegCount.get() + " of " + numRegs + " regions processed so far");

http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 261a9aa..1ba3af2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
 import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
@@ -2727,6 +2729,10 @@ public final class ProtobufUtil {
     return tableNames;
   }
 
+  public static BackupProtos.BackupType toProtoBackupType(BackupType type) {
+    return BackupProtos.BackupType.valueOf(type.name());
+  }
+
   /**
    * Convert a protocol buffer CellVisibility to a client CellVisibility
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/b37cc760/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 99e993d..5babfb8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Action;
 import org.apache.hadoop.hbase.client.Admin;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
@@ -1268,6 +1270,22 @@ public final class RequestConverter {
     return builder.build();
   }
 
+  public static BackupTablesRequest buildBackupTablesRequest(
+      final BackupType type, List<TableName> tableList, String targetRootDir, final
int workers,
+      final long bandwidth) {
+    BackupTablesRequest.Builder builder = BackupTablesRequest.newBuilder();
+    builder.setType(ProtobufUtil.toProtoBackupType(type));
+    builder.setTargetRootDir(targetRootDir);
+    builder.setWorkers(workers);
+    builder.setBandwidth(bandwidth);
+    if (tableList != null) {
+      for (TableName table : tableList) {
+        builder.addTables(ProtobufUtil.toProtoTableName(table));
+      }
+    }
+    return builder.build();
+  }
+
   /**
    * Creates a protocol buffer GetSchemaAlterStatusRequest
    *


Mime
View raw message