You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/09/08 20:53:06 UTC

hbase git commit: HBASE-16588 Remove ConnectionFactory#createConnection from backup / restore server code

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 4c9ecdf86 -> 76d7acdfa


HBASE-16588 Remove ConnectionFactory#createConnection from backup / restore server code


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

Branch: refs/heads/HBASE-7912
Commit: 76d7acdfa369a6be5a5fb3a24604461ad0479781
Parents: 4c9ecdf
Author: tedyu <yu...@gmail.com>
Authored: Thu Sep 8 13:51:18 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Sep 8 13:51:18 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/RestoreDriver.java      | 64 ++++++++++----------
 .../hadoop/hbase/backup/impl/BackupManager.java |  4 +-
 .../backup/impl/RestoreTablesProcedure.java     | 20 +++---
 .../backup/master/FullTableBackupProcedure.java |  6 +-
 .../master/IncrementalTableBackupProcedure.java |  9 ++-
 .../hbase/backup/util/BackupServerUtil.java     | 10 ++-
 .../hbase/backup/util/RestoreServerUtil.java    | 22 +++----
 7 files changed, 71 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
index 3fd0c33..32ef218 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -125,37 +125,37 @@ public class RestoreDriver extends AbstractHBaseTool {
     String backupId = remainArgs[1];
     String tables = null;
     String tableMapping = null;
-    // Check backup set
-    if (cmd.hasOption(OPTION_SET)) {
-      String setName = cmd.getOptionValue(OPTION_SET);
-      try{
-        tables = getTablesForSet(setName, conf);       
-      } catch(IOException e){
-        System.out.println("ERROR: "+ e.getMessage()+" for setName="+setName);
-        return -2;
-      }
-      if (tables == null) {
-        System.out.println("ERROR: Backup set '" + setName
-        + "' is either empty or does not exist");
-        return -3;
-      }
-      tableMapping = (remainArgs.length > 2) ? remainArgs[2] : null;
-    } else {
-      tables = remainArgs[2];    
-      tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
-    }    
-
-    TableName[] sTableArray = BackupServerUtil.parseTableNames(tables);
-    TableName[] tTableArray = BackupServerUtil.parseTableNames(tableMapping);
-
-    if (sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length)) {
-      System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
-      System.out.println(USAGE);
-      return -4;
-    }
-
     try (final Connection conn = ConnectionFactory.createConnection(conf);
         BackupAdmin client = conn.getAdmin().getBackupAdmin();) {
+      // Check backup set
+      if (cmd.hasOption(OPTION_SET)) {
+        String setName = cmd.getOptionValue(OPTION_SET);
+        try{
+          tables = getTablesForSet(conn, setName, conf);
+        } catch(IOException e){
+          System.out.println("ERROR: "+ e.getMessage()+" for setName="+setName);
+          return -2;
+        }
+        if (tables == null) {
+          System.out.println("ERROR: Backup set '" + setName
+              + "' is either empty or does not exist");
+          return -3;
+        }
+        tableMapping = (remainArgs.length > 2) ? remainArgs[2] : null;
+      } else {
+        tables = remainArgs[2];    
+        tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
+      }    
+
+      TableName[] sTableArray = BackupServerUtil.parseTableNames(tables);
+      TableName[] tTableArray = BackupServerUtil.parseTableNames(tableMapping);
+
+      if (sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length)){
+        System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+        System.out.println(USAGE);
+        return -4;
+      }
+
       client.restore(RestoreServerUtil.createRestoreRequest(backupRootDir, backupId, check,
           sTableArray, tTableArray, isOverwrite));
     } catch (Exception e){
@@ -165,9 +165,9 @@ public class RestoreDriver extends AbstractHBaseTool {
     return 0;
   }
 
-  private String getTablesForSet(String name, Configuration conf) throws IOException {
-    try (final Connection conn = ConnectionFactory.createConnection(conf);
-        final BackupSystemTable table = new BackupSystemTable(conn)) {
+  private String getTablesForSet(Connection conn, String name, Configuration conf)
+      throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
       List<TableName> tables = table.describeBackupSet(name);
       if (tables == null) return null;
       return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
index bea466b..a8d21bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -80,13 +80,13 @@ public class BackupManager implements Closeable {
    * @param conf configuration
    * @throws IOException exception
    */
-  public BackupManager(Configuration conf) throws IOException {
+  public BackupManager(Connection conn, Configuration conf) throws IOException {
     if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
       throw new BackupException("HBase backup is not enabled. Check your " +
           HConstants.BACKUP_ENABLE_KEY + " setting.");
     }
     this.conf = conf;
-    this.conn = ConnectionFactory.createConnection(conf);
+    this.conn = conn;
     this.systemTable = new BackupSystemTable(conn);
      
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesProcedure.java
index 4ec232b..214ad80 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesProcedure.java
@@ -139,14 +139,15 @@ public class RestoreTablesProcedure
 
   /**
    * Restore operation handle each backupImage in iterator
+   * @param conn the Connection
    * @param it: backupImage iterator - ascending
    * @param sTable: table to be restored
    * @param tTable: table to be restored to
+   * @param truncateIfExists truncate table if it exists
    * @throws IOException exception
    */
-  private void restoreImages(Iterator<BackupImage> it, TableName sTable,
-      TableName tTable, boolean truncateIfExists)
-      throws IOException {
+  private void restoreImages(Connection conn, Iterator<BackupImage> it, TableName sTable,
+      TableName tTable, boolean truncateIfExists) throws IOException {
 
     // First image MUST be image of a FULL backup
     BackupImage image = it.next();
@@ -180,7 +181,7 @@ public class RestoreTablesProcedure
     if (manifest.getType() == BackupType.FULL || converted) {
       LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from "
           + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString());
-      restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable,
+      restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable,
         converted, truncateIfExists, lastIncrBackupId);
     } else { // incremental Backup
       throw new IOException("Unexpected backup type " + image.getType());
@@ -194,7 +195,7 @@ public class RestoreTablesProcedure
       String[] sarr = new String[logDirList.size()];
       logDirList.toArray(sarr);
       Path[] paths = org.apache.hadoop.util.StringUtils.stringToPath(sarr);
-      restoreTool.incrementalRestoreTable(tableBackupPath, paths, new TableName[] { sTable },
+      restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable },
         new TableName[] { tTable }, lastIncrBackupId);
     }
     LOG.info(sTable + " has been successfully restored to " + tTable);
@@ -202,13 +203,15 @@ public class RestoreTablesProcedure
 
   /**
    * Restore operation. Stage 2: resolved Backup Image dependency
+   * @param conn the Connection
    * @param backupManifestMap : tableName,  Manifest
    * @param sTableArray The array of tables to be restored
    * @param tTableArray The array of mapping tables to restore to
+   * @param isOverwrite overwrite
    * @return set of BackupImages restored
    * @throws IOException exception
    */
-  private void restoreStage(HashMap<TableName, BackupManifest> backupManifestMap,
+  private void restoreStage(Connection conn, HashMap<TableName, BackupManifest> backupManifestMap,
       TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException {
     TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
     boolean truncateIfExists = isOverwrite;
@@ -224,7 +227,7 @@ public class RestoreTablesProcedure
         list.addAll(depList);
         TreeSet<BackupImage> restoreList = new TreeSet<BackupImage>(list);
         LOG.debug("need to clear merged Image. to be implemented in future jira");
-        restoreImages(restoreList.iterator(), table, tTableArray[i], truncateIfExists);
+        restoreImages(conn, restoreList.iterator(), table, tTableArray[i], truncateIfExists);
         restoreImageSet.addAll(restoreList);
 
         if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
@@ -273,7 +276,8 @@ public class RestoreTablesProcedure
           Path rootPath = new Path(targetRootDir);
           HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
             backupId);
-          restoreStage(backupManifestMap, sTableArray, tTableArray, isOverwrite);
+          restoreStage(env.getMasterServices().getConnection(), backupManifestMap, sTableArray,
+              tTableArray, isOverwrite);
 
           return Flow.NO_MORE_STATE;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
index 1862d97..f2d2615 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
@@ -96,7 +96,8 @@ public class FullTableBackupProcedure
   public FullTableBackupProcedure(final MasterProcedureEnv env,
       final String backupId, List<TableName> tableList, String targetRootDir, final int workers,
       final long bandwidth) throws IOException {
-    backupManager = new BackupManager(env.getMasterConfiguration());
+    backupManager = new BackupManager(env.getMasterServices().getConnection(),
+        env.getMasterConfiguration());
     this.backupId = backupId;
     this.tableList = tableList;
     this.targetRootDir = targetRootDir;
@@ -489,7 +490,8 @@ public class FullTableBackupProcedure
     }
     if (backupManager == null) {
       try {
-        backupManager = new BackupManager(env.getMasterConfiguration());
+        backupManager = new BackupManager(env.getMasterServices().getConnection(),
+            env.getMasterConfiguration());
       } catch (IOException ioe) {
         setFailure("full backup", ioe);
         return Flow.NO_MORE_STATE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
index db8e582..0a54e8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
@@ -79,7 +79,8 @@ public class IncrementalTableBackupProcedure
       final String backupId,
       List<TableName> tableList, String targetRootDir, final int workers,
       final long bandwidth) throws IOException {
-    backupManager = new BackupManager(env.getMasterConfiguration());
+    backupManager = new BackupManager(env.getMasterServices().getConnection(),
+        env.getMasterConfiguration());
     this.backupId = backupId;
     this.tableList = tableList;
     this.targetRootDir = targetRootDir;
@@ -203,7 +204,8 @@ public class IncrementalTableBackupProcedure
     }
     if (backupManager == null) {
       try {
-        backupManager = new BackupManager(env.getMasterConfiguration());
+        backupManager = new BackupManager(env.getMasterServices().getConnection(),
+            env.getMasterConfiguration());
       } catch (IOException ioe) {
         setFailure("incremental backup", ioe);
       }
@@ -233,7 +235,8 @@ public class IncrementalTableBackupProcedure
         case INCREMENTAL_COPY:
           try {
             // copy out the table and region info files for each table
-            BackupServerUtil.copyTableRegionInfo(backupContext, conf);
+            BackupServerUtil.copyTableRegionInfo(env.getMasterServices().getConnection(),
+                backupContext, conf);
             incrementalCopy(backupContext);
             // Save list of WAL files copied
             backupManager.recordWALFiles(backupContext.getIncrBackupFileList());

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
index 37c8d65..5e0f3c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
@@ -150,22 +150,20 @@ public final class BackupServerUtil {
   /**
    * copy out Table RegionInfo into incremental backup image need to consider move this logic into
    * HBackupFileSystem
+   * @conn the Connection
    * @param backupContext backup context
    * @param conf configuration
    * @throws IOException exception
    * @throws InterruptedException exception
    */
-  public static void copyTableRegionInfo(BackupInfo backupContext, Configuration conf)
-      throws IOException, InterruptedException {
-
+  public static void copyTableRegionInfo(Connection conn, BackupInfo backupContext,
+      Configuration conf) throws IOException, InterruptedException {
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     // for each table in the table set, copy out the table info and region 
     // info files in the correct directory structure
-    try (Connection conn = ConnectionFactory.createConnection(conf); 
-        Admin admin = conn.getAdmin()) {
-
+    try (Admin admin = conn.getAdmin()) {
       for (TableName table : backupContext.getTables()) {
 
         if(!admin.tableExists(table)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/76d7acdf/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
index e179a73..007ca9e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
@@ -186,8 +187,8 @@ public class RestoreServerUtil {
    * @param incrBackupId incremental backup Id
    * @throws IOException exception
    */
-  public void incrementalRestoreTable(Path tableBackupPath, Path[] logDirs, TableName[] tableNames,
-      TableName[] newTableNames, String incrBackupId) throws IOException {
+  public void incrementalRestoreTable(Connection conn, Path tableBackupPath, Path[] logDirs,
+      TableName[] tableNames, TableName[] newTableNames, String incrBackupId) throws IOException {
 
     if (tableNames.length != newTableNames.length) {
       throw new IOException("Number of source tables and target tables does not match!");
@@ -196,10 +197,9 @@ public class RestoreServerUtil {
 
     // for incremental backup image, expect the table already created either by user or previous
     // full backup. Here, check that all new tables exists
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        Admin admin = conn.getAdmin()) {
+    try (Admin admin = conn.getAdmin()) {
       for (TableName tableName : newTableNames) {
-        if (!admin.tableExists(tableName)) {
+        if (!MetaTableAccessor.tableExists(conn, tableName)) {
           admin.close();
           throw new IOException("HBase table " + tableName
             + " does not exist. Create the table first, e.g. by restoring a full backup.");
@@ -241,9 +241,10 @@ public class RestoreServerUtil {
     }
   }
 
-  public void fullRestoreTable(Path tableBackupPath, TableName tableName, TableName newTableName,
-      boolean converted, boolean truncateIfExists, String lastIncrBackupId) throws IOException {
-    restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted, truncateIfExists,
+  public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName,
+      TableName newTableName, boolean converted, boolean truncateIfExists, String lastIncrBackupId)
+          throws IOException {
+    restoreTableAndCreate(conn, tableName, newTableName, tableBackupPath, converted, truncateIfExists,
         lastIncrBackupId);
   }
 
@@ -362,7 +363,7 @@ public class RestoreServerUtil {
     return null;
   }
 
-  private void restoreTableAndCreate(TableName tableName, TableName newTableName,
+  private void restoreTableAndCreate(Connection conn, TableName tableName, TableName newTableName,
       Path tableBackupPath, boolean converted, boolean truncateIfExists, String lastIncrBackupId)
           throws IOException {
     if (newTableName == null || newTableName.equals("")) {
@@ -376,8 +377,7 @@ public class RestoreServerUtil {
       LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId);
     }
 
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        HBaseAdmin hbadmin = (HBaseAdmin) conn.getAdmin();) {
+    try (HBaseAdmin hbadmin = (HBaseAdmin) conn.getAdmin();) {
       if (tableDescriptor == null) {
         Path tableSnapshotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
         if (fileSys.exists(tableSnapshotPath)) {