diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java index da197f1..3871b03 100644 --- hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java +++ hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.backup.BackupRequest; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HBaseAdmin; @@ -162,7 +162,7 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase { List tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2); HBaseAdmin admin = null; admin = (HBaseAdmin) conn.getAdmin(); - BackupAdmin client = new HBaseBackupAdmin(util.getConnection()); + BackupAdmin client = new BackupAdminImpl(util.getConnection()); BackupRequest request = new BackupRequest(); request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java index 45981f9..834245f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -176,7 +176,7 @@ public class BackupInfo implements Comparable { this.addTables(tables); if (type == BackupType.INCREMENTAL) { - setHlogTargetDir(BackupClientUtil.getLogBackupDir(targetRootDir, backupId)); + setHLogTargetDir(BackupClientUtil.getLogBackupDir(targetRootDir, backupId)); } this.startTs = 0; @@ -220,10 +220,6 @@ public class BackupInfo implements Comparable { this.tableSetTimestampMap = tableSetTimestampMap; } - public String getHlogTargetDir() { - return hlogTargetDir; - } - public void setType(BackupType type) { this.type = type; } @@ -355,7 +351,7 @@ public class BackupInfo implements Comparable { return targetRootDir; } - public void setHlogTargetDir(String hlogTagetDir) { + public void setHLogTargetDir(String hlogTagetDir) { this.hlogTargetDir = hlogTagetDir; } @@ -488,7 +484,7 @@ public class BackupInfo implements Comparable { context.setState(BackupInfo.BackupState.valueOf(proto.getState().name())); } - context.setHlogTargetDir(BackupClientUtil.getLogBackupDir(proto.getTargetRootDir(), + context.setHLogTargetDir(BackupClientUtil.getLogBackupDir(proto.getTargetRootDir(), proto.getBackupId())); if (proto.hasPhase()) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java index a130a9b..08145fb 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -66,7 +66,7 @@ public class HBackupFileSystem { */ public static String getTableBackupDir(String backupRootDir, String backupId, TableName tableName) { - return backupRootDir + Path.SEPARATOR+ backupId + Path.SEPARATOR + + return backupRootDir + Path.SEPARATOR+ backupId + Path.SEPARATOR + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString() + Path.SEPARATOR ; } @@ -84,8 +84,8 @@ public class HBackupFileSystem { return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName)); } - - public static List loadRegionInfos(TableName tableName, + + public static List loadRegionInfos(TableName tableName, Path backupRootPath, String backupId, Configuration conf) throws IOException { Path backupTableRoot = getTableBackupPath(tableName, backupRootPath, backupId); @@ -100,11 +100,11 @@ public class HBackupFileSystem { infos.add(info); } } - + Collections.sort(infos); return infos; } - + /** * Given the backup root dir and the backup id, return the log file location for an incremental * backup. @@ -125,6 +125,7 @@ public class HBackupFileSystem { Path backupRootPath, String backupId) throws IOException { Path manifestPath = new Path(getTableBackupPath(tableName, backupRootPath, backupId), BackupManifest.MANIFEST_FILE_NAME); + FileSystem fs = backupRootPath.getFileSystem(conf); if (!fs.exists(manifestPath)) { // check log dir for incremental backup case @@ -134,8 +135,8 @@ public class HBackupFileSystem { if (!fs.exists(manifestPath)) { String errorMsg = "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for " + - backupId + " in " + backupRootPath.toString() + - ". Did " + backupId + " correspond to previously taken backup ?"; + backupId + ". File " + manifestPath + + " does not exists. Did " + backupId + " correspond to previously taken backup ?"; throw new IOException(errorMsg); } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java index 6b04944..1ca512e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupManager; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.util.BackupServerUtil; import org.apache.hadoop.hbase.backup.util.LogUtils; import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; @@ -110,7 +110,7 @@ public class RestoreDriver extends AbstractHBaseTool implements BackupRestoreCon String tableMapping = cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null; try (final Connection conn = ConnectionFactory.createConnection(conf); - BackupAdmin client = new HBaseBackupAdmin(conn);) { + BackupAdmin client = new BackupAdminImpl(conn);) { // Check backup set if (cmd.hasOption(OPTION_SET)) { String setName = cmd.getOptionValue(OPTION_SET); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java new file mode 100644 index 0000000..0e094d5 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -0,0 +1,556 @@ +/** + * 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.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.util.BackupClientUtil; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +import com.google.common.collect.Lists; + +/** + * The administrative API implementation for HBase Backup . Create an instance from + * {@link BackupAdminImpl(Connection)} and call {@link #close()} afterwards. + *

BackupAdmin can be used to create backups, restore data from backups and for + * other backup-related operations. + * + * @see Admin + * @since 2.0 + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving + +public class BackupAdminImpl implements BackupAdmin { + private static final Log LOG = LogFactory.getLog(BackupAdminImpl.class); + + private final Connection conn; + + public BackupAdminImpl(Connection conn) { + this.conn = conn; + } + + @Override + public void close() throws IOException { + if (conn != null) { + conn.close(); + } + } + + @Override + public BackupInfo getBackupInfo(String backupId) throws IOException { + BackupInfo backupInfo = null; + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + backupInfo = table.readBackupInfo(backupId); + return backupInfo; + } + } + + @Override + public int getProgress(String backupId) throws IOException { + BackupInfo backupInfo = null; + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (backupId == null) { + ArrayList recentSessions = table.getBackupContexts(BackupState.RUNNING); + if (recentSessions.isEmpty()) { + LOG.warn("No ongoing sessions found."); + return -1; + } + // else show status for ongoing session + // must be one maximum + return recentSessions.get(0).getProgress(); + } else { + + backupInfo = table.readBackupInfo(backupId); + if (backupInfo != null) { + return backupInfo.getProgress(); + } else { + LOG.warn("No information found for backupID=" + backupId); + return -1; + } + } + } + } + + @Override + public int deleteBackups(String[] backupIds) throws IOException { + // TODO: requires FT, failure will leave system + // in non-consistent state + // see HBASE-15227 + + int totalDeleted = 0; + Map> allTablesMap = new HashMap>(); + + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + for (int i = 0; i < backupIds.length; i++) { + BackupInfo info = sysTable.readBackupInfo(backupIds[i]); + if (info != null) { + String rootDir = info.getTargetRootDir(); + HashSet allTables = allTablesMap.get(rootDir); + if (allTables == null) { + allTables = new HashSet(); + allTablesMap.put(rootDir, allTables); + } + allTables.addAll(info.getTableNames()); + totalDeleted += deleteBackup(backupIds[i], sysTable); + } + } + finalizeDelete(allTablesMap, sysTable); + } + return totalDeleted; + } + + /** + * Updates incremental backup set for every backupRoot + * @param tablesMap - Map [backupRoot: Set] + * @param table - backup system table + * @throws IOException + */ + + private void finalizeDelete(Map> tablesMap, BackupSystemTable table) + throws IOException { + for (String backupRoot : tablesMap.keySet()) { + Set incrTableSet = table.getIncrementalBackupTableSet(backupRoot); + Map> tableMap = + table.getBackupHistoryForTableSet(incrTableSet, backupRoot); + for(Map.Entry> entry: tableMap.entrySet()) { + if(entry.getValue() == null) { + // No more backups for a table + incrTableSet.remove(entry.getKey()); + } + } + if (!incrTableSet.isEmpty()) { + table.addIncrementalBackupTableSet(incrTableSet, backupRoot); + } else { // empty + table.deleteIncrementalBackupTableSet(backupRoot); + } + } + } + + /** + * Delete single backup and all related backups + * Algorithm: + * + * Backup type: FULL or INCREMENTAL + * Is this last backup session for table T: YES or NO + * For every table T from table list 'tables': + * if(FULL, YES) deletes only physical data (PD) + * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo, until + * we either reach the most recent backup for T in the system or FULL backup which + * includes T + * if(INCREMENTAL, YES) deletes only physical data (PD) + * if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images + * between last FULL backup, which is older than the backup being deleted and the next + * FULL backup (if exists) or last one for a particular table T and removes T from list + * of backup tables. + * @param backupId - backup id + * @param sysTable - backup system table + * @return total - number of deleted backup images + * @throws IOException + */ + private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { + + BackupInfo backupInfo = sysTable.readBackupInfo(backupId); + + int totalDeleted = 0; + if (backupInfo != null) { + LOG.info("Deleting backup " + backupInfo.getBackupId() + " ..."); + BackupClientUtil.cleanupBackupData(backupInfo, conn.getConfiguration()); + // List of tables in this backup; + List tables = backupInfo.getTableNames(); + long startTime = backupInfo.getStartTs(); + for (TableName tn : tables) { + boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime); + if (isLastBackupSession) { + continue; + } + // else + List affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable); + for (BackupInfo info : affectedBackups) { + if (info.equals(backupInfo)) { + continue; + } + removeTableFromBackupImage(info, tn, sysTable); + } + } + LOG.debug("Delete backup info "+ backupInfo.getBackupId()); + + sysTable.deleteBackupInfo(backupInfo.getBackupId()); + LOG.info("Delete backup " + backupInfo.getBackupId() + " completed."); + totalDeleted++; + } else { + LOG.warn("Delete backup failed: no information found for backupID=" + backupId); + } + return totalDeleted; + } + + private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable) + throws IOException { + List tables = info.getTableNames(); + LOG.debug("Remove "+ tn +" from " + info.getBackupId() + " tables=" + + info.getTableListAsString()); + if (tables.contains(tn)) { + tables.remove(tn); + + if (tables.isEmpty()) { + LOG.debug("Delete backup info "+ info.getBackupId()); + + sysTable.deleteBackupInfo(info.getBackupId()); + BackupClientUtil.cleanupBackupData(info, conn.getConfiguration()); + } else { + info.setTables(tables); + sysTable.updateBackupInfo(info); + // Now, clean up directory for table + cleanupBackupDir(info, tn, conn.getConfiguration()); + } + } + } + + private List getAffectedBackupInfos(BackupInfo backupInfo, TableName tn, + BackupSystemTable table) throws IOException { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn); + long ts = backupInfo.getStartTs(); + List list = new ArrayList(); + List history = table.getBackupHistory(backupInfo.getTargetRootDir()); + // Scan from most recent to backupInfo + // break when backupInfo reached + for (BackupInfo info : history) { + if (info.getStartTs() == ts) { + break; + } + List tables = info.getTableNames(); + if (tables.contains(tn)) { + BackupType bt = info.getType(); + if (bt == BackupType.FULL) { + // Clear list if we encounter FULL backup + list.clear(); + } else { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn + + " added " + info.getBackupId() + " tables=" + info.getTableListAsString()); + list.add(info); + } + } + } + return list; + } + + + + /** + * Clean up the data at target directory + * @throws IOException + */ + private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) + throws IOException { + try { + // clean up the data at target directory + String targetDir = backupInfo.getTargetRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupInfo.getBackupId()); + return; + } + + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf); + + Path targetDirPath = + new Path(BackupClientUtil.getTableBackupDir(backupInfo.getTargetRootDir(), + backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table + + "at " + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); + throw e1; + } + } + + private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) + throws IOException { + List history = table.getBackupHistory(); + for (BackupInfo info : history) { + List tables = info.getTableNames(); + if (!tables.contains(tn)) { + continue; + } + if (info.getStartTs() <= startTime) { + return true; + } else { + return false; + } + } + return false; + } + + @Override + public List getHistory(int n) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + if (history.size() <= n) return history; + List list = new ArrayList(); + for (int i = 0; i < n; i++) { + list.add(history.get(i)); + } + return list; + } + } + + @Override + public List getHistory(int n, BackupInfo.Filter ... filters) throws IOException { + if (filters.length == 0) return getHistory(n); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + List result = new ArrayList(); + for(BackupInfo bi: history) { + if(result.size() == n) break; + boolean passed = true; + for(int i=0; i < filters.length; i++) { + if(!filters[i].apply(bi)) { + passed = false; + break; + } + } + if(passed) { + result.add(bi); + } + } + return result; + } + } + + @Override + public List listBackupSets() throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.listBackupSets(); + List bslist = new ArrayList(); + for (String s : list) { + List tables = table.describeBackupSet(s); + if (tables != null) { + bslist.add(new BackupSet(s, tables)); + } + } + return bslist; + } + } + + @Override + public BackupSet getBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.describeBackupSet(name); + if (list == null) return null; + return new BackupSet(name, list); + } + } + + @Override + public boolean deleteBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (table.describeBackupSet(name) == null) { + return false; + } + table.deleteBackupSet(name); + return true; + } + } + + @Override + public void addToBackupSet(String name, TableName[] tables) throws IOException { + String[] tableNames = new String[tables.length]; + try (final BackupSystemTable table = new BackupSystemTable(conn); + final Admin admin = conn.getAdmin();) { + for (int i = 0; i < tables.length; i++) { + tableNames[i] = tables[i].getNameAsString(); + if (!admin.tableExists(TableName.valueOf(tableNames[i]))) { + throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist"); + } + } + table.addToBackupSet(name, tableNames); + LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name + + "' backup set"); + } + } + + @Override + public void removeFromBackupSet(String name, String[] tables) throws IOException { + LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'"); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + table.removeFromBackupSet(name, tables); + LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + + "' completed."); + } + } + + @Override + public void restore(RestoreRequest request) throws IOException { + if (request.isCheck()) { + HashMap backupManifestMap = new HashMap<>(); + // check and load backup image manifest for the tables + Path rootPath = new Path(request.getBackupRootDir()); + String backupId = request.getBackupId(); + TableName[] sTableArray = request.getFromTables(); + HBackupFileSystem.checkImageManifestExist(backupManifestMap, + sTableArray, conn.getConfiguration(), rootPath, backupId); + + // Check and validate the backup image and its dependencies + + if (RestoreServerUtil.validate(backupManifestMap, conn.getConfiguration())) { + LOG.info("Checking backup images: ok"); + } else { + String errMsg = "Some dependencies are missing for restore"; + LOG.error(errMsg); + throw new IOException(errMsg); + } + + } + // Execute restore request + new RestoreTablesClient(conn, request).execute(); + } + + @Override + public Future restoreAsync(RestoreRequest request) throws IOException { + throw new UnsupportedOperationException("Asynchronous restore is not supported yet"); + } + + @Override + public String backupTables(final BackupRequest request) throws IOException { + String setName = request.getBackupSetName(); + BackupType type = request.getBackupType(); + String targetRootDir = request.getTargetRootDir(); + List tableList = request.getTableList(); + + String backupId = + (setName == null || setName.length() == 0 ? BackupRestoreConstants.BACKUPID_PREFIX + : setName + "_") + EnvironmentEdgeManager.currentTime(); + if (type == BackupType.INCREMENTAL) { + Set incrTableSet = null; + try (BackupSystemTable table = new BackupSystemTable(conn)) { + incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + } + + if (incrTableSet.isEmpty()) { + System.err.println("Incremental backup table set contains no table.\n" + + "Use 'backup create full' or 'backup stop' to \n " + + "change the tables covered by incremental backup."); + throw new IOException("No table covered by incremental backup."); + } + + tableList.removeAll(incrTableSet); + if (!tableList.isEmpty()) { + String extraTables = StringUtils.join(tableList, ","); + System.err.println("Some tables (" + extraTables + ") haven't gone through full backup"); + throw new IOException("Perform full backup on " + extraTables + " first, " + + "then retry the command"); + } + System.out.println("Incremental backup for the following table set: " + incrTableSet); + tableList = Lists.newArrayList(incrTableSet); + } + if (tableList != null && !tableList.isEmpty()) { + for (TableName table : tableList) { + String targetTableBackupDir = + HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); + Path targetTableBackupDirPath = new Path(targetTableBackupDir); + FileSystem outputFs = + FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration()); + if (outputFs.exists(targetTableBackupDirPath)) { + throw new IOException("Target backup directory " + targetTableBackupDir + + " exists already."); + } + } + ArrayList nonExistingTableList = null; + try (Admin admin = conn.getAdmin();) { + for (TableName tableName : tableList) { + if (!admin.tableExists(tableName)) { + if (nonExistingTableList == null) { + nonExistingTableList = new ArrayList<>(); + } + nonExistingTableList.add(tableName); + } + } + } + if (nonExistingTableList != null) { + if (type == BackupType.INCREMENTAL) { + System.err.println("Incremental backup table set contains non-exising table: " + + nonExistingTableList); + // Update incremental backup set + tableList = excludeNonExistingTables(tableList, nonExistingTableList); + } else { + // Throw exception only in full mode - we try to backup non-existing table + throw new IOException("Non-existing tables found in the table list: " + + nonExistingTableList); + } + } + } + + // update table list + request.setTableList(tableList); + + if (type == BackupType.FULL) { + new FullTableBackupClient(conn, backupId, request).execute(); + } else { + new IncrementalTableBackupClient(conn, backupId, request).execute(); + } + return backupId; + } + + + private List excludeNonExistingTables(List tableList, + List nonExistingTableList) { + + for (TableName table : nonExistingTableList) { + tableList.remove(table); + } + return tableList; + } + + @Override + public Future backupTablesAsync(final BackupRequest userRequest) throws IOException { + throw new UnsupportedOperationException("Asynchronous backup is not supported yet"); + } + +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 436f419..a1f3e25 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -229,7 +229,7 @@ public final class BackupCommands implements BackupRestoreConstants { Integer.parseInt(cmdline.getOptionValue(OPTION_WORKERS)) : -1; try (Connection conn = ConnectionFactory.createConnection(getConf()); - HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) { + BackupAdminImpl admin = new BackupAdminImpl(conn);) { BackupRequest request = new BackupRequest(); request.setBackupType(BackupType.valueOf(args[1].toUpperCase())) .setTableList(tables != null?Lists.newArrayList(BackupClientUtil.parseTableNames(tables)): null) @@ -461,7 +461,7 @@ public final class BackupCommands implements BackupRestoreConstants { System.arraycopy(args, 1, backupIds, 0, backupIds.length); Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); try (final Connection conn = ConnectionFactory.createConnection(conf); - HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) { + BackupAdminImpl admin = new BackupAdminImpl(conn);) { int deleted = admin.deleteBackups(args); System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length); } @@ -491,7 +491,7 @@ public final class BackupCommands implements BackupRestoreConstants { } Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); try (final Connection conn = ConnectionFactory.createConnection(conf); - HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) { + BackupAdminImpl admin = new BackupAdminImpl(conn);) { // TODO cancel backup } } @@ -666,7 +666,7 @@ public final class BackupCommands implements BackupRestoreConstants { // does not expect any args Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create(); try(final Connection conn = ConnectionFactory.createConnection(conf); - HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){ + BackupAdminImpl admin = new BackupAdminImpl(conn);){ List list = admin.listBackupSets(); for(BackupSet bs: list){ System.out.println(bs); @@ -701,7 +701,7 @@ public final class BackupCommands implements BackupRestoreConstants { String setName = args[2]; Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create(); try(final Connection conn = ConnectionFactory.createConnection(conf); - final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){ + final BackupAdminImpl admin = new BackupAdminImpl(conn);){ boolean result = admin.deleteBackupSet(setName); if(result){ System.out.println("Delete set "+setName+" OK."); @@ -721,7 +721,7 @@ public final class BackupCommands implements BackupRestoreConstants { String[] tables = args[3].split(","); Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create(); try(final Connection conn = ConnectionFactory.createConnection(conf); - final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){ + final BackupAdminImpl admin = new BackupAdminImpl(conn);){ admin.removeFromBackupSet(setName, tables); } } @@ -739,7 +739,7 @@ public final class BackupCommands implements BackupRestoreConstants { } Configuration conf = getConf() != null? getConf():HBaseConfiguration.create(); try(final Connection conn = ConnectionFactory.createConnection(conf); - final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){ + final BackupAdminImpl admin = new BackupAdminImpl(conn);){ admin.addToBackupSet(setName, tableNames); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java index af715d4..10afd65 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -25,10 +25,6 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -52,8 +48,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - /** * Handles backup requests on server-side, creates backup context records in hbase:backup * to keep track backup. The timestamps kept in hbase:backup table will be used for future @@ -63,11 +57,10 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; @InterfaceStability.Evolving public class BackupManager implements Closeable { private static final Log LOG = LogFactory.getLog(BackupManager.class); - private Configuration conf = null; - private BackupInfo backupContext = null; - private ExecutorService pool = null; - private BackupSystemTable systemTable; - private final Connection conn; + protected Configuration conf = null; + protected BackupInfo backupContext = null; + protected BackupSystemTable systemTable; + protected final Connection conn; /** * Backup manager constructor. @@ -173,11 +166,7 @@ public class BackupManager implements Closeable { */ @Override public void close() { - // currently, we shutdown now for all ongoing back handlers, we may need to do something like - // record the failed list somewhere later - if (this.pool != null) { - this.pool.shutdownNow(); - } + if (systemTable != null) { try { systemTable.close(); @@ -185,13 +174,6 @@ public class BackupManager implements Closeable { LOG.error(e); } } - if (conn != null) { - try { - conn.close(); - } catch (IOException e) { - LOG.error(e); - } - } } /** @@ -270,15 +252,6 @@ public class BackupManager implements Closeable { + ". Can not launch new backup until no ongoing backup remains."); throw new BackupException("There is ongoing backup."); } - - // Initialize thread pools - int nrThreads = this.conf.getInt("hbase.backup.threads.max", 1); - ThreadFactoryBuilder builder = new ThreadFactoryBuilder(); - builder.setNameFormat("BackupHandler-%1$d"); - this.pool = - new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS, - new LinkedBlockingQueue(), builder.build()); - ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); } public void setBackupContext(BackupInfo backupContext) { @@ -309,11 +282,14 @@ public class BackupManager implements Closeable { ArrayList allHistoryList = getBackupHistory(true); for (BackupInfo backup : allHistoryList) { - BackupImage image = - new BackupImage(backup.getBackupId(), backup.getType(), - backup.getTargetRootDir(), - backup.getTableNames(), backup.getStartTs(), backup - .getEndTs()); + + BackupImage.Builder builder = BackupImage.newBuilder(); + + BackupImage image = builder.withBackupId(backup.getBackupId()). + withType(backup.getType()).withRootDir(backup.getTargetRootDir()). + withTableList(backup.getTableNames()).withStartTime(backup.getStartTs()). + withCompleteTime(backup.getEndTs()).build(); + // add the full backup image as an ancestor until the last incremental backup if (backup.getType().equals(BackupType.FULL)) { // check the backup image coverage, if previous image could be covered by the newer ones, @@ -328,10 +304,9 @@ public class BackupManager implements Closeable { // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing // incremental backup if (BackupManifest.canCoverImage(ancestors, image)) { - LOG.debug("Met the backup boundary of the current table set. " - + "The root full backup images for the current backup scope:"); + LOG.debug("Met the backup boundary of the current table set:"); for (BackupImage image1 : ancestors) { - LOG.debug(" BackupId: " + image1.getBackupId() + ", Backup directory: " + LOG.debug(" BackupID=" + image1.getBackupId() + ", BackupDir=" + image1.getRootDir()); } } else { @@ -345,9 +320,10 @@ public class BackupManager implements Closeable { BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage(); ancestors.add(lastIncrImage); - LOG.debug("Last dependent incremental backup image information:"); - LOG.debug(" Token: " + lastIncrImage.getBackupId()); - LOG.debug(" Backup directory: " + lastIncrImage.getRootDir()); + LOG.debug("Last dependent incremental backup image: " + + "{BackupID=" + lastIncrImage.getBackupId()+"," + + "BackupDir=" + lastIncrImage.getRootDir()+"}" + ); } } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java index 9ea9f9d..00f55b7 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -35,7 +35,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo; @@ -66,6 +65,49 @@ public class BackupManifest { public static class BackupImage implements Comparable { + static class Builder { + BackupImage image; + + Builder() { + image = new BackupImage(); + } + + Builder withBackupId(String backupId) { + image.setBackupId(backupId); + return this; + } + + Builder withType(BackupType type) { + image.setType(type); + return this; + } + + Builder withRootDir(String rootDir) { + image.setRootDir(rootDir); + return this; + } + + Builder withTableList(List tableList) { + image.setTableList(tableList); + return this; + } + + Builder withStartTime(long startTime) { + image.setStartTs(startTime); + return this; + } + + Builder withCompleteTime(long completeTime) { + image.setCompleteTs(completeTime); + return this; + } + + BackupImage build() { + return image; + } + + } + private String backupId; private BackupType type; private String rootDir; @@ -74,12 +116,16 @@ public class BackupManifest { private long completeTs; private ArrayList ancestors; private HashMap> incrTimeRanges; - + + static Builder newBuilder() { + return new Builder(); + } + public BackupImage() { super(); } - public BackupImage(String backupId, BackupType type, String rootDir, + private BackupImage(String backupId, BackupType type, String rootDir, List tableList, long startTs, long completeTs) { this.backupId = backupId; this.type = type; @@ -99,9 +145,9 @@ public class BackupManifest { for(HBaseProtos.TableName tn : tableListList) { tableList.add(ProtobufUtil.toTableName(tn)); } - + List ancestorList = im.getAncestorsList(); - + BackupType type = im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL: BackupType.INCREMENTAL; @@ -135,17 +181,17 @@ public class BackupManifest { builder.addAncestors(im.toProto()); } } - - setIncrementalTimestampMap(builder); + + setIncrementalTimestampMap(builder); return builder.build(); } - - private static HashMap> + + private static HashMap> loadIncrementalTimestampMap(BackupProtos.BackupImage proto) { List list = proto.getTstMapList(); - - HashMap> incrTimeRanges = + + HashMap> incrTimeRanges = new HashMap>(); if(list == null || list.size() == 0) return incrTimeRanges; for(BackupProtos.TableServerTimestamp tst: list){ @@ -164,7 +210,7 @@ public class BackupManifest { return incrTimeRanges; } - + private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) { if (this.incrTimeRanges == null) { return; @@ -183,14 +229,14 @@ public class BackupManifest { ServerName sn = ServerName.parseServerName(s); snBuilder.setHostName(sn.getHostname()); snBuilder.setPort(sn.getPort()); - stBuilder.setServer(snBuilder.build()); + stBuilder.setServer(snBuilder.build()); stBuilder.setTimestamp(entry2.getValue()); tstBuilder.addServerTimestamp(stBuilder.build()); } builder.addTstMap(tstBuilder.build()); } - } - + } + public String getBackupId() { return backupId; } @@ -312,86 +358,44 @@ public class BackupManifest { } } - // hadoop hbase configuration - protected Configuration config = null; - - // backup root directory - private String rootDir = null; - // backup image directory private String tableBackupDir = null; - - // backup log directory if this is an incremental backup - private String logBackupDir = null; - - // backup token - private String backupId; - - // backup type, full or incremental - private BackupType type; - - // the table list for the backup - private ArrayList tableList; - - // actual start timestamp of the backup process - private long startTs; - - // actual complete timestamp of the backup process - private long completeTs; - - // the region server timestamp for tables: - // > - private Map> incrTimeRanges; - - // dependency of this backup, including all the dependent images to do PIT recovery - //private Map dependency; private BackupImage backupImage; - + /** * Construct manifest for a ongoing backup. - * @param backupCtx The ongoing backup context + * @param backup The ongoing backup info */ - public BackupManifest(BackupInfo backupCtx) { - this.backupId = backupCtx.getBackupId(); - this.type = backupCtx.getType(); - this.rootDir = backupCtx.getTargetRootDir(); - if (this.type == BackupType.INCREMENTAL) { - this.logBackupDir = backupCtx.getHLogTargetDir(); - } - this.startTs = backupCtx.getStartTs(); - this.completeTs = backupCtx.getEndTs(); - this.loadTableList(backupCtx.getTableNames()); - this.backupImage = new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs, - this.completeTs); + public BackupManifest(BackupInfo backup) { + + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = builder.withBackupId(backup.getBackupId()). + withType(backup.getType()).withRootDir(backup.getTargetRootDir()). + withTableList(backup.getTableNames()).withStartTime(backup.getStartTs()). + withCompleteTime(backup.getEndTs()).build(); } - - + + /** * Construct a table level manifest for a backup of the named table. - * @param backupCtx The ongoing backup context + * @param backup The ongoing backup session info */ - public BackupManifest(BackupInfo backupCtx, TableName table) { - this.backupId = backupCtx.getBackupId(); - this.type = backupCtx.getType(); - this.rootDir = backupCtx.getTargetRootDir(); - this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir(); - if (this.type == BackupType.INCREMENTAL) { - this.logBackupDir = backupCtx.getHLogTargetDir(); - } - this.startTs = backupCtx.getStartTs(); - this.completeTs = backupCtx.getEndTs(); + public BackupManifest(BackupInfo backup, TableName table) { + this.tableBackupDir = backup.getBackupStatus(table).getTargetDir(); List tables = new ArrayList(); tables.add(table); - this.loadTableList(tables); - this.backupImage = new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs, - this.completeTs); + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = builder.withBackupId(backup.getBackupId()). + withType(backup.getType()).withRootDir(backup.getTargetRootDir()). + withTableList(tables).withStartTime(backup.getStartTs()). + withCompleteTime(backup.getEndTs()).build(); } /** * Construct manifest from a backup directory. * @param conf configuration * @param backupPath backup path - * @throws IOException + * @throws IOException */ public BackupManifest(Configuration conf, Path backupPath) throws IOException { @@ -413,8 +417,6 @@ public class BackupManifest { // It could be the backup log dir where there is also a manifest file stored. // This variable's purpose is to keep the correct and original location so // that we can store/persist it. - this.tableBackupDir = backupPath.toString(); - this.config = fs.getConf(); try { FileStatus[] subFiles = BackupClientUtil.listStatus(fs, backupPath, null); @@ -438,23 +440,6 @@ public class BackupManifest { throw new BackupException(e); } this.backupImage = BackupImage.fromProto(proto); - // Here the parameter backupDir is where the manifest file is. - // There should always be a manifest file under: - // backupRootDir/namespace/table/backupId/.backup.manifest - this.rootDir = backupPath.getParent().getParent().getParent().toString(); - - Path p = backupPath.getParent(); - if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) { - this.rootDir = p.getParent().toString(); - } else { - this.rootDir = p.getParent().getParent().toString(); - } - this.backupId = this.backupImage.getBackupId(); - this.startTs = this.backupImage.getStartTs(); - this.completeTs = this.backupImage.getCompleteTs(); - this.type = this.backupImage.getType(); - this.tableList = (ArrayList)this.backupImage.getTableNames(); - this.incrTimeRanges = this.backupImage.getIncrTimeRanges(); LOG.debug("Loaded manifest instance from manifest file: " + BackupClientUtil.getPath(subFile.getPath())); return; @@ -469,39 +454,15 @@ public class BackupManifest { } public BackupType getType() { - return type; - } - - public void setType(BackupType type) { - this.type = type; - } - - /** - * Loads table list. - * @param tableList Table list - */ - private void loadTableList(List tableList) { - - this.tableList = this.getTableList(); - if (this.tableList.size() > 0) { - this.tableList.clear(); - } - for (int i = 0; i < tableList.size(); i++) { - this.tableList.add(tableList.get(i)); - } - - LOG.debug(tableList.size() + " tables exist in table set."); + return backupImage.getType(); } /** * Get the table set of this image. * @return The table set list */ - public ArrayList getTableList() { - if (this.tableList == null) { - this.tableList = new ArrayList(); - } - return this.tableList; + public List getTableList() { + return backupImage.getTableNames(); } /** @@ -512,15 +473,16 @@ public class BackupManifest { public void store(Configuration conf) throws BackupException { byte[] data = backupImage.toProto().toByteArray(); // write the file, overwrite if already exist + String logBackupDir = BackupClientUtil.getLogBackupDir(backupImage.getRootDir(), + backupImage.getBackupId()); Path manifestFilePath = - new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir)) + new Path(new Path((tableBackupDir != null ? tableBackupDir : logBackupDir)) ,MANIFEST_FILE_NAME); - try { - FSDataOutputStream out = - manifestFilePath.getFileSystem(conf).create(manifestFilePath, true); + try ( FSDataOutputStream out = + manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);) + { out.write(data); - out.close(); - } catch (IOException e) { + } catch (IOException e) { throw new BackupException(e.getMessage()); } @@ -548,15 +510,11 @@ public class BackupManifest { * @param incrTimestampMap timestamp map */ public void setIncrTimestampMap(HashMap> incrTimestampMap) { - this.incrTimeRanges = incrTimestampMap; this.backupImage.setIncrTimeRanges(incrTimestampMap); } public Map> getIncrTimestampMap() { - if (this.incrTimeRanges == null) { - this.incrTimeRanges = new HashMap>(); - } - return this.incrTimeRanges; + return backupImage.getIncrTimeRanges(); } /** @@ -692,18 +650,21 @@ public class BackupManifest { LOG.debug("Full image set can cover image " + image.getBackupId()); return true; } - + public BackupInfo toBackupInfo() { BackupInfo info = new BackupInfo(); - info.setType(type); - TableName[] tables = new TableName[tableList.size()]; - info.addTables(getTableList().toArray(tables)); - info.setBackupId(backupId); - info.setStartTs(startTs); - info.setTargetRootDir(rootDir); - if(type == BackupType.INCREMENTAL) { - info.setHlogTargetDir(logBackupDir); + info.setType(backupImage.getType()); + List list = backupImage.getTableNames(); + TableName[] tables = new TableName[list.size()]; + info.addTables(list.toArray(tables)); + info.setBackupId(backupImage.getBackupId()); + info.setStartTs(backupImage.getStartTs()); + info.setTargetRootDir(backupImage.getRootDir()); + if(backupImage.getType() == BackupType.INCREMENTAL) { + + info.setHLogTargetDir(BackupClientUtil.getLogBackupDir(backupImage.getRootDir(), + backupImage.getBackupId())); } return info; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index e04c36e..b16f45f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -32,6 +32,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -90,7 +91,8 @@ public final class BackupSystemTable implements Closeable { @Override public String toString() { - return "/" + backupRoot + "/" + backupId + "/" + walFile; + return Path.SEPARATOR + backupRoot + + Path.SEPARATOR + backupId + Path.SEPARATOR + walFile; } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java index f5911b4..557ee08 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java @@ -45,7 +45,8 @@ import org.apache.hadoop.hbase.util.Bytes; @InterfaceAudience.Private @InterfaceStability.Evolving -public final class BackupSystemTableHelper { + +final class BackupSystemTableHelper { /** * hbase:backup schema: diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index f1f09cc..e068ede 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -19,239 +19,40 @@ package org.apache.hadoop.hbase.backup.impl; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupCopyTask; import org.apache.hadoop.hbase.backup.BackupInfo; import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.BackupRequest; -import org.apache.hadoop.hbase.backup.BackupRestoreConstants; import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory; import org.apache.hadoop.hbase.backup.BackupType; -import org.apache.hadoop.hbase.backup.HBackupFileSystem; -import org.apache.hadoop.hbase.backup.impl.BackupException; -import org.apache.hadoop.hbase.backup.impl.BackupManifest; -import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; import org.apache.hadoop.hbase.backup.util.BackupClientUtil; import org.apache.hadoop.hbase.backup.util.BackupServerUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; @InterfaceAudience.Private -public class FullTableBackupClient { +public class FullTableBackupClient extends TableBackupClient{ private static final Log LOG = LogFactory.getLog(FullTableBackupClient.class); - private Configuration conf; - private Connection conn; - private String backupId; - private List tableList; - HashMap newTimestamps = null; - - private BackupManager backupManager; - private BackupInfo backupContext; - - public FullTableBackupClient() { - // Required by the Procedure framework to create the procedure on replay - } public FullTableBackupClient(final Connection conn, final String backupId, BackupRequest request) throws IOException { - backupManager = new BackupManager(conn, conn.getConfiguration()); - this.backupId = backupId; - this.tableList = request.getTableList(); - this.conn = conn; - this.conf = conn.getConfiguration(); - backupContext = - backupManager.createBackupContext(backupId, BackupType.FULL, tableList, - request.getTargetRootDir(), - request.getWorkers(), request.getBandwidth()); - if (tableList == null || tableList.isEmpty()) { - this.tableList = new ArrayList<>(backupContext.getTables()); - } - } - - /** - * Begin the overall backup. - * @param backupContext backup context - * @throws IOException exception - */ - static void beginBackup(BackupManager backupManager, BackupInfo backupContext) throws IOException { - backupManager.setBackupContext(backupContext); - // set the start timestamp of the overall backup - long startTs = EnvironmentEdgeManager.currentTime(); - backupContext.setStartTs(startTs); - // set overall backup status: ongoing - backupContext.setState(BackupState.RUNNING); - LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + "."); - - backupManager.updateBackupInfo(backupContext); - if (LOG.isDebugEnabled()) { - LOG.debug("Backup session " + backupContext.getBackupId() + " has been started."); - } - } - - private static String getMessage(Exception e) { - String msg = e.getMessage(); - if (msg == null || msg.equals("")) { - msg = e.getClass().getName(); - } - return msg; - } - - /** - * Delete HBase snapshot for backup. - * @param backupCtx backup context - * @throws Exception exception - */ - private static void - deleteSnapshot(final Connection conn, BackupInfo backupCtx, Configuration conf) - throws IOException { - LOG.debug("Trying to delete snapshot for full backup."); - for (String snapshotName : backupCtx.getSnapshotNames()) { - if (snapshotName == null) { - continue; - } - LOG.debug("Trying to delete snapshot: " + snapshotName); - - try (Admin admin = conn.getAdmin();) { - admin.deleteSnapshot(snapshotName); - } catch (IOException ioe) { - LOG.debug("when deleting snapshot " + snapshotName, ioe); - } - LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupCtx.getBackupId() - + " succeeded."); - } + super(conn, backupId, request); } - /** - * Clean up directories with prefix "exportSnapshot-", which are generated when exporting - * snapshots. - * @throws IOException exception - */ - private static void cleanupExportSnapshotLog(Configuration conf) throws IOException { - FileSystem fs = FSUtils.getCurrentFileSystem(conf); - Path stagingDir = - new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory() - .toString())); - FileStatus[] files = FSUtils.listStatus(fs, stagingDir); - if (files == null) { - return; - } - for (FileStatus file : files) { - if (file.getPath().getName().startsWith("exportSnapshot-")) { - LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); - if (FSUtils.delete(fs, file.getPath(), true) == false) { - LOG.warn("Can not delete " + file.getPath()); - } - } - } - } - - /** - * Clean up the uncompleted data at target directory if the ongoing backup has already entered the - * copy phase. - */ - static void cleanupTargetDir(BackupInfo backupContext, Configuration conf) { - try { - // clean up the uncompleted data at target directory if the ongoing backup has already entered - // the copy phase - LOG.debug("Trying to cleanup up target dir. Current backup phase: " - + backupContext.getPhase()); - if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY) - || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY) - || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) { - FileSystem outputFs = - FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf); - - // now treat one backup as a transaction, clean up data that has been partially copied at - // table level - for (TableName table : backupContext.getTables()) { - Path targetDirPath = - new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(), - backupContext.getBackupId(), table)); - if (outputFs.delete(targetDirPath, true)) { - LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString() - + " done."); - } else { - LOG.info("No data has been copied to " + targetDirPath.toString() + "."); - } - - Path tableDir = targetDirPath.getParent(); - FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir); - if (backups == null || backups.length == 0) { - outputFs.delete(tableDir, true); - LOG.debug(tableDir.toString() + " is empty, remove it."); - } - } - } - - } catch (IOException e1) { - LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at " - + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); - } - } - /** - * Fail the overall backup. - * @param backupContext backup context - * @param e exception - * @throws Exception exception - */ - static void failBackup(Connection conn, BackupInfo backupContext, BackupManager backupManager, - Exception e, String msg, BackupType type, Configuration conf) throws IOException { - LOG.error(msg + getMessage(e), e); - // If this is a cancel exception, then we've already cleaned. - - // set the failure timestamp of the overall backup - backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); - - // set failure message - backupContext.setFailedMsg(e.getMessage()); - - // set overall backup status: failed - backupContext.setState(BackupState.FAILED); - - // compose the backup failed data - String backupFailedData = - "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs() - + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase() - + ",failedmessage=" + backupContext.getFailedMsg(); - LOG.error(backupFailedData); - - backupManager.updateBackupInfo(backupContext); - - // if full backup, then delete HBase snapshots if there already are snapshots taken - // and also clean up export snapshot log files if exist - if (type == BackupType.FULL) { - deleteSnapshot(conn, backupContext, conf); - cleanupExportSnapshotLog(conf); - } - - // clean up the uncompleted data at target directory if the ongoing backup has already entered - // the copy phase - // For incremental backup, DistCp logs will be cleaned with the targetDir. - cleanupTargetDir(backupContext, conf); - - LOG.info("Backup " + backupContext.getBackupId() + " failed."); - } /** * Do snapshot copy. @@ -298,162 +99,10 @@ public class FullTableBackupClient { } /** - * Add manifest for the current backup. The manifest is stored within the table backup directory. - * @param backupContext The current backup context - * @throws IOException exception - * @throws BackupException exception - */ - private static void addManifest(BackupInfo backupContext, BackupManager backupManager, - BackupType type, Configuration conf) throws IOException, BackupException { - // set the overall backup phase : store manifest - backupContext.setPhase(BackupPhase.STORE_MANIFEST); - - BackupManifest manifest; - - // Since we have each table's backup in its own directory structure, - // we'll store its manifest with the table directory. - for (TableName table : backupContext.getTables()) { - manifest = new BackupManifest(backupContext, table); - ArrayList ancestors = backupManager.getAncestors(backupContext, table); - for (BackupImage image : ancestors) { - manifest.addDependentImage(image); - } - - if (type == BackupType.INCREMENTAL) { - // We'll store the log timestamps for this table only in its manifest. - HashMap> tableTimestampMap = - new HashMap>(); - tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table)); - manifest.setIncrTimestampMap(tableTimestampMap); - ArrayList ancestorss = backupManager.getAncestors(backupContext); - for (BackupImage image : ancestorss) { - manifest.addDependentImage(image); - } - } - manifest.store(conf); - } - - // For incremental backup, we store a overall manifest in - // /WALs/ - // This is used when created the next incremental backup - if (type == BackupType.INCREMENTAL) { - manifest = new BackupManifest(backupContext); - // set the table region server start and end timestamps for incremental backup - manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap()); - ArrayList ancestors = backupManager.getAncestors(backupContext); - for (BackupImage image : ancestors) { - manifest.addDependentImage(image); - } - manifest.store(conf); - } - } - - /** - * Get backup request meta data dir as string. - * @param backupContext backup context - * @return meta data dir - */ - private static String obtainBackupMetaDataStr(BackupInfo backupContext) { - StringBuffer sb = new StringBuffer(); - sb.append("type=" + backupContext.getType() + ",tablelist="); - for (TableName table : backupContext.getTables()) { - sb.append(table + ";"); - } - if (sb.lastIndexOf(";") > 0) { - sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); - } - sb.append(",targetRootDir=" + backupContext.getTargetRootDir()); - - return sb.toString(); - } - - /** - * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying - * hlogs. - * @throws IOException exception - */ - private static void cleanupDistCpLog(BackupInfo backupContext, Configuration conf) - throws IOException { - Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent(); - FileSystem fs = FileSystem.get(rootPath.toUri(), conf); - FileStatus[] files = FSUtils.listStatus(fs, rootPath); - if (files == null) { - return; - } - for (FileStatus file : files) { - if (file.getPath().getName().startsWith("_distcp_logs")) { - LOG.debug("Delete log files of DistCp: " + file.getPath().getName()); - FSUtils.delete(fs, file.getPath(), true); - } - } - } - - /** - * Complete the overall backup. - * @param backupContext backup context - * @throws Exception exception - */ - static void completeBackup(final Connection conn, BackupInfo backupContext, - BackupManager backupManager, BackupType type, Configuration conf) throws IOException { - // set the complete timestamp of the overall backup - backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); - // set overall backup status: complete - backupContext.setState(BackupState.COMPLETE); - backupContext.setProgress(100); - // add and store the manifest for the backup - addManifest(backupContext, backupManager, type, conf); - - // after major steps done and manifest persisted, do convert if needed for incremental backup - /* in-fly convert code here, provided by future jira */ - LOG.debug("in-fly convert code here, provided by future jira"); - - // compose the backup complete data - String backupCompleteData = - obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs() - + ",completets=" + backupContext.getEndTs() + ",bytescopied=" - + backupContext.getTotalBytesCopied(); - if (LOG.isDebugEnabled()) { - LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData); - } - backupManager.updateBackupInfo(backupContext); - - // when full backup is done: - // - delete HBase snapshot - // - clean up directories with prefix "exportSnapshot-", which are generated when exporting - // snapshots - if (type == BackupType.FULL) { - deleteSnapshot(conn, backupContext, conf); - cleanupExportSnapshotLog(conf); - } else if (type == BackupType.INCREMENTAL) { - cleanupDistCpLog(backupContext, conf); - } - - LOG.info("Backup " + backupContext.getBackupId() + " completed."); - } - - /** - * Wrap a SnapshotDescription for a target table. - * @param table table - * @return a SnapshotDescription especially for backup. - */ - static SnapshotDescription wrapSnapshotDescription(TableName tableName, String snapshotName) { - // Mock a SnapshotDescription from backupContext to call SnapshotManager function, - // Name it in the format "snapshot__" - HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder(); - builder.setTable(tableName.getNameAsString()); - builder.setName(snapshotName); - HBaseProtos.SnapshotDescription backupSnapshot = builder.build(); - - LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName() - + " from backupContext to request snapshot for backup."); - - return backupSnapshot; - } - - /** * Backup request execution * @throws IOException */ + @Override public void execute() throws IOException { try (Admin admin = conn.getAdmin();) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java deleted file mode 100644 index f5b3ea7..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java +++ /dev/null @@ -1,556 +0,0 @@ -/** - * 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.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Future; - -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.BackupInfo; -import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; -import org.apache.hadoop.hbase.backup.BackupAdmin; -import org.apache.hadoop.hbase.backup.BackupRequest; -import org.apache.hadoop.hbase.backup.BackupRestoreConstants; -import org.apache.hadoop.hbase.backup.BackupType; -import org.apache.hadoop.hbase.backup.HBackupFileSystem; -import org.apache.hadoop.hbase.backup.RestoreRequest; -import org.apache.hadoop.hbase.backup.util.BackupClientUtil; -import org.apache.hadoop.hbase.backup.util.BackupSet; -import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - -import com.google.common.collect.Lists; - -/** - * The administrative API implementation for HBase Backup . Create an instance from - * {@link HBaseBackupAdmin(Connection)} and call {@link #close()} afterwards. - *

BackupAdmin can be used to create backups, restore data from backups and for - * other backup-related operations. - * - * @see Admin - * @since 2.0 - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving - -public class HBaseBackupAdmin implements BackupAdmin { - private static final Log LOG = LogFactory.getLog(HBaseBackupAdmin.class); - - private final Connection conn; - - public HBaseBackupAdmin(Connection conn) { - this.conn = conn; - } - - @Override - public void close() throws IOException { - } - - @Override - public BackupInfo getBackupInfo(String backupId) throws IOException { - BackupInfo backupInfo = null; - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - backupInfo = table.readBackupInfo(backupId); - return backupInfo; - } - } - - @Override - public int getProgress(String backupId) throws IOException { - BackupInfo backupInfo = null; - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - if (backupId == null) { - ArrayList recentSessions = table.getBackupContexts(BackupState.RUNNING); - if (recentSessions.isEmpty()) { - LOG.warn("No ongoing sessions found."); - return -1; - } - // else show status for ongoing session - // must be one maximum - return recentSessions.get(0).getProgress(); - } else { - - backupInfo = table.readBackupInfo(backupId); - if (backupInfo != null) { - return backupInfo.getProgress(); - } else { - LOG.warn("No information found for backupID=" + backupId); - return -1; - } - } - } - } - - @Override - public int deleteBackups(String[] backupIds) throws IOException { - // TODO: requires FT, failure will leave system - // in non-consistent state - // see HBASE-15227 - - int totalDeleted = 0; - Map> allTablesMap = new HashMap>(); - - try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { - for (int i = 0; i < backupIds.length; i++) { - BackupInfo info = sysTable.readBackupInfo(backupIds[i]); - if (info != null) { - String rootDir = info.getTargetRootDir(); - HashSet allTables = allTablesMap.get(rootDir); - if (allTables == null) { - allTables = new HashSet(); - allTablesMap.put(rootDir, allTables); - } - allTables.addAll(info.getTableNames()); - totalDeleted += deleteBackup(backupIds[i], sysTable); - } - } - finalizeDelete(allTablesMap, sysTable); - } - return totalDeleted; - } - - /** - * Updates incremental backup set for every backupRoot - * @param tablesMap - Map [backupRoot: Set] - * @param table - backup system table - * @throws IOException - */ - - private void finalizeDelete(Map> tablesMap, BackupSystemTable table) - throws IOException { - for (String backupRoot : tablesMap.keySet()) { - Set incrTableSet = table.getIncrementalBackupTableSet(backupRoot); - Map> tableMap = - table.getBackupHistoryForTableSet(incrTableSet, backupRoot); - for(Map.Entry> entry: tableMap.entrySet()) { - if(entry.getValue() == null) { - // No more backups for a table - incrTableSet.remove(entry.getKey()); - } - } - if (!incrTableSet.isEmpty()) { - table.addIncrementalBackupTableSet(incrTableSet, backupRoot); - } else { // empty - table.deleteIncrementalBackupTableSet(backupRoot); - } - } - } - - /** - * Delete single backup and all related backups - * Algorithm: - * - * Backup type: FULL or INCREMENTAL - * Is this last backup session for table T: YES or NO - * For every table T from table list 'tables': - * if(FULL, YES) deletes only physical data (PD) - * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo, until - * we either reach the most recent backup for T in the system or FULL backup which - * includes T - * if(INCREMENTAL, YES) deletes only physical data (PD) - * if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images - * between last FULL backup, which is older than the backup being deleted and the next - * FULL backup (if exists) or last one for a particular table T and removes T from list - * of backup tables. - * @param backupId - backup id - * @param sysTable - backup system table - * @return total - number of deleted backup images - * @throws IOException - */ - private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { - - BackupInfo backupInfo = sysTable.readBackupInfo(backupId); - - int totalDeleted = 0; - if (backupInfo != null) { - LOG.info("Deleting backup " + backupInfo.getBackupId() + " ..."); - BackupClientUtil.cleanupBackupData(backupInfo, conn.getConfiguration()); - // List of tables in this backup; - List tables = backupInfo.getTableNames(); - long startTime = backupInfo.getStartTs(); - for (TableName tn : tables) { - boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime); - if (isLastBackupSession) { - continue; - } - // else - List affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable); - for (BackupInfo info : affectedBackups) { - if (info.equals(backupInfo)) { - continue; - } - removeTableFromBackupImage(info, tn, sysTable); - } - } - LOG.debug("Delete backup info "+ backupInfo.getBackupId()); - - sysTable.deleteBackupInfo(backupInfo.getBackupId()); - LOG.info("Delete backup " + backupInfo.getBackupId() + " completed."); - totalDeleted++; - } else { - LOG.warn("Delete backup failed: no information found for backupID=" + backupId); - } - return totalDeleted; - } - - private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable) - throws IOException { - List tables = info.getTableNames(); - LOG.debug("Remove "+ tn +" from " + info.getBackupId() + " tables=" + - info.getTableListAsString()); - if (tables.contains(tn)) { - tables.remove(tn); - - if (tables.isEmpty()) { - LOG.debug("Delete backup info "+ info.getBackupId()); - - sysTable.deleteBackupInfo(info.getBackupId()); - BackupClientUtil.cleanupBackupData(info, conn.getConfiguration()); - } else { - info.setTables(tables); - sysTable.updateBackupInfo(info); - // Now, clean up directory for table - cleanupBackupDir(info, tn, conn.getConfiguration()); - } - } - } - - private List getAffectedBackupInfos(BackupInfo backupInfo, TableName tn, - BackupSystemTable table) throws IOException { - LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn); - long ts = backupInfo.getStartTs(); - List list = new ArrayList(); - List history = table.getBackupHistory(backupInfo.getTargetRootDir()); - // Scan from most recent to backupInfo - // break when backupInfo reached - for (BackupInfo info : history) { - if (info.getStartTs() == ts) { - break; - } - List tables = info.getTableNames(); - if (tables.contains(tn)) { - BackupType bt = info.getType(); - if (bt == BackupType.FULL) { - // Clear list if we encounter FULL backup - list.clear(); - } else { - LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn - + " added " + info.getBackupId() + " tables=" + info.getTableListAsString()); - list.add(info); - } - } - } - return list; - } - - - - /** - * Clean up the data at target directory - * @throws IOException - */ - private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) - throws IOException { - try { - // clean up the data at target directory - String targetDir = backupInfo.getTargetRootDir(); - if (targetDir == null) { - LOG.warn("No target directory specified for " + backupInfo.getBackupId()); - return; - } - - FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf); - - Path targetDirPath = - new Path(BackupClientUtil.getTableBackupDir(backupInfo.getTargetRootDir(), - backupInfo.getBackupId(), table)); - if (outputFs.delete(targetDirPath, true)) { - LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); - } else { - LOG.info("No data has been found in " + targetDirPath.toString() + "."); - } - - } catch (IOException e1) { - LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table - + "at " + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); - throw e1; - } - } - - private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) - throws IOException { - List history = table.getBackupHistory(); - for (BackupInfo info : history) { - List tables = info.getTableNames(); - if (!tables.contains(tn)) { - continue; - } - if (info.getStartTs() <= startTime) { - return true; - } else { - return false; - } - } - return false; - } - - @Override - public List getHistory(int n) throws IOException { - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - List history = table.getBackupHistory(); - if (history.size() <= n) return history; - List list = new ArrayList(); - for (int i = 0; i < n; i++) { - list.add(history.get(i)); - } - return list; - } - } - - @Override - public List getHistory(int n, BackupInfo.Filter ... filters) throws IOException { - if (filters.length == 0) return getHistory(n); - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - List history = table.getBackupHistory(); - List result = new ArrayList(); - for(BackupInfo bi: history) { - if(result.size() == n) break; - boolean passed = true; - for(int i=0; i < filters.length; i++) { - if(!filters[i].apply(bi)) { - passed = false; - break; - } - } - if(passed) { - result.add(bi); - } - } - return result; - } - } - - @Override - public List listBackupSets() throws IOException { - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - List list = table.listBackupSets(); - List bslist = new ArrayList(); - for (String s : list) { - List tables = table.describeBackupSet(s); - if (tables != null) { - bslist.add(new BackupSet(s, tables)); - } - } - return bslist; - } - } - - @Override - public BackupSet getBackupSet(String name) throws IOException { - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - List list = table.describeBackupSet(name); - if (list == null) return null; - return new BackupSet(name, list); - } - } - - @Override - public boolean deleteBackupSet(String name) throws IOException { - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - if (table.describeBackupSet(name) == null) { - return false; - } - table.deleteBackupSet(name); - return true; - } - } - - @Override - public void addToBackupSet(String name, TableName[] tables) throws IOException { - String[] tableNames = new String[tables.length]; - try (final BackupSystemTable table = new BackupSystemTable(conn); - final Admin admin = conn.getAdmin();) { - for (int i = 0; i < tables.length; i++) { - tableNames[i] = tables[i].getNameAsString(); - if (!admin.tableExists(TableName.valueOf(tableNames[i]))) { - throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist"); - } - } - table.addToBackupSet(name, tableNames); - LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name - + "' backup set"); - } - } - - @Override - public void removeFromBackupSet(String name, String[] tables) throws IOException { - LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'"); - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - table.removeFromBackupSet(name, tables); - LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name - + "' completed."); - } - } - - @Override - public void restore(RestoreRequest request) throws IOException { - if (request.isCheck()) { - HashMap backupManifestMap = new HashMap<>(); - // check and load backup image manifest for the tables - Path rootPath = new Path(request.getBackupRootDir()); - String backupId = request.getBackupId(); - TableName[] sTableArray = request.getFromTables(); - HBackupFileSystem.checkImageManifestExist(backupManifestMap, - sTableArray, conn.getConfiguration(), rootPath, backupId); - - // Check and validate the backup image and its dependencies - - if (RestoreServerUtil.validate(backupManifestMap, conn.getConfiguration())) { - LOG.info("Checking backup images: ok"); - } else { - String errMsg = "Some dependencies are missing for restore"; - LOG.error(errMsg); - throw new IOException(errMsg); - } - - } - // Execute restore request - new RestoreTablesClient(conn, request).execute(); - } - - @Override - public Future restoreAsync(RestoreRequest request) throws IOException { - // TBI - return null; - } - - @Override - public String backupTables(final BackupRequest request) throws IOException { - String setName = request.getBackupSetName(); - BackupType type = request.getBackupType(); - String targetRootDir = request.getTargetRootDir(); - List tableList = request.getTableList(); - - String backupId = - (setName == null || setName.length() == 0 ? BackupRestoreConstants.BACKUPID_PREFIX - : setName + "_") + EnvironmentEdgeManager.currentTime(); - if (type == BackupType.INCREMENTAL) { - Set incrTableSet = null; - try (BackupSystemTable table = new BackupSystemTable(conn)) { - incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); - } - - if (incrTableSet.isEmpty()) { - System.err.println("Incremental backup table set contains no table.\n" - + "Use 'backup create full' or 'backup stop' to \n " - + "change the tables covered by incremental backup."); - throw new IOException("No table covered by incremental backup."); - } - - tableList.removeAll(incrTableSet); - if (!tableList.isEmpty()) { - String extraTables = StringUtils.join(tableList, ","); - System.err.println("Some tables (" + extraTables + ") haven't gone through full backup"); - throw new IOException("Perform full backup on " + extraTables + " first, " - + "then retry the command"); - } - System.out.println("Incremental backup for the following table set: " + incrTableSet); - tableList = Lists.newArrayList(incrTableSet); - } - if (tableList != null && !tableList.isEmpty()) { - for (TableName table : tableList) { - String targetTableBackupDir = - HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); - Path targetTableBackupDirPath = new Path(targetTableBackupDir); - FileSystem outputFs = - FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration()); - if (outputFs.exists(targetTableBackupDirPath)) { - throw new IOException("Target backup directory " + targetTableBackupDir - + " exists already."); - } - } - ArrayList nonExistingTableList = null; - try (Admin admin = conn.getAdmin();) { - for (TableName tableName : tableList) { - if (!admin.tableExists(tableName)) { - if (nonExistingTableList == null) { - nonExistingTableList = new ArrayList<>(); - } - nonExistingTableList.add(tableName); - } - } - } - if (nonExistingTableList != null) { - if (type == BackupType.INCREMENTAL) { - System.err.println("Incremental backup table set contains non-exising table: " - + nonExistingTableList); - // Update incremental backup set - tableList = excludeNonExistingTables(tableList, nonExistingTableList); - } else { - // Throw exception only in full mode - we try to backup non-existing table - throw new IOException("Non-existing tables found in the table list: " - + nonExistingTableList); - } - } - } - - // update table list - request.setTableList(tableList); - - if (type == BackupType.FULL) { - new FullTableBackupClient(conn, backupId, request).execute(); - } else { - new IncrementalTableBackupClient(conn, backupId, request).execute(); - } - return backupId; - } - - - private List excludeNonExistingTables(List tableList, - List nonExistingTableList) { - - for (TableName table : nonExistingTableList) { - tableList.remove(table); - } - return tableList; - } - - @Override - public Future backupTablesAsync(final BackupRequest userRequest) throws IOException { - // TBI - return null; - } - -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 1a5e5d4..ab5c0c1 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -53,16 +53,11 @@ import org.apache.hadoop.hbase.wal.DefaultWALProvider; */ @InterfaceAudience.Public @InterfaceStability.Evolving -public class IncrementalBackupManager { +public class IncrementalBackupManager extends BackupManager{ public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class); - // parent manager - private final BackupManager backupManager; - private final Configuration conf; - - public IncrementalBackupManager(BackupManager bm) { - this.backupManager = bm; - this.conf = bm.getConf(); + public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException { + super(conn, conf); } /** @@ -79,14 +74,13 @@ public class IncrementalBackupManager { HashMap newTimestamps; HashMap previousTimestampMins; - String savedStartCode = backupManager.readBackupStartCode(); + String savedStartCode = readBackupStartCode(); // key: tableName // value: - HashMap> previousTimestampMap = - backupManager.readLogTimestampMap(); + HashMap> previousTimestampMap = readLogTimestampMap(); - previousTimestampMins = BackupServerUtil.getRSLogTimestampMins(previousTimestampMap); + previousTimestampMins = BackupServerUtil.getRSLogTimestampMins(previousTimestampMap); if (LOG.isDebugEnabled()) { LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId()); @@ -102,19 +96,19 @@ public class IncrementalBackupManager { LOG.info("Execute roll log procedure for incremental backup ..."); HashMap props = new HashMap(); props.put("backupRoot", backupContext.getTargetRootDir()); - + try(Admin admin = conn.getAdmin();) { - - admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); } - newTimestamps = backupManager.readRegionServerLastLogRollResult(); + newTimestamps = readRegionServerLastLogRollResult(); logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode); - List logFromSystemTable = - getLogFilesFromBackupSystem(previousTimestampMins, - newTimestamps, backupManager.getBackupContext().getTargetRootDir()); + List logFromSystemTable = + getLogFilesFromBackupSystem(previousTimestampMins, + newTimestamps, getBackupContext().getTargetRootDir()); addLogsFromBackupSystemToContext(logFromSystemTable); logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable); @@ -126,14 +120,14 @@ public class IncrementalBackupManager { private List excludeAlreadyBackedUpWALs(List logList, List logFromSystemTable) { - + List backupedWALList = toWALList(logFromSystemTable); logList.removeAll(backupedWALList); return logList; } private List toWALList(List logFromSystemTable) { - + List list = new ArrayList(logFromSystemTable.size()); for(WALItem item : logFromSystemTable){ list.add(item.getWalFile()); @@ -149,7 +143,7 @@ public class IncrementalBackupManager { String backupId = item.getBackupId(); String relWALPath = backupId + Path.SEPARATOR+walFileName; walFiles.add(relWALPath); - } + } } @@ -164,7 +158,7 @@ public class IncrementalBackupManager { private List getLogFilesFromBackupSystem(HashMap olderTimestamps, HashMap newestTimestamps, String backupRoot) throws IOException { List logFiles = new ArrayList(); - Iterator it = backupManager.getWALFilesFromBackupSystem(); + Iterator it = getWALFilesFromBackupSystem(); while (it.hasNext()) { WALItem item = it.next(); String rootDir = item.getBackupRoot(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 8acace0..1da5b19 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -26,7 +26,6 @@ import java.util.List; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; @@ -44,33 +43,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; @InterfaceAudience.Private -public class IncrementalTableBackupClient { +public class IncrementalTableBackupClient extends TableBackupClient { private static final Log LOG = LogFactory.getLog(IncrementalTableBackupClient.class); - private Configuration conf; - private Connection conn; - //private String backupId; - HashMap newTimestamps = null; - - private String backupId; - private BackupManager backupManager; - private BackupInfo backupContext; - - public IncrementalTableBackupClient() { - // Required by the Procedure framework to create the procedure on replay - } public IncrementalTableBackupClient(final Connection conn, final String backupId, BackupRequest request) - throws IOException { - - this.conn = conn; - this.conf = conn.getConfiguration(); - backupManager = new BackupManager(conn, conf); - this.backupId = backupId; - backupContext = - backupManager.createBackupContext(backupId, BackupType.INCREMENTAL, request.getTableList(), - request.getTargetRootDir(), request.getWorkers(), (int) request.getBandwidth()); + throws IOException + { + super(conn, backupId, request); } private List filterMissingFiles(List incrBackupFileList) throws IOException { @@ -172,19 +153,19 @@ public class IncrementalTableBackupClient { return list; } + @Override public void execute() throws IOException { // case PREPARE_INCREMENTAL: - FullTableBackupClient.beginBackup(backupManager, backupContext); + beginBackup(backupManager, backupContext); LOG.debug("For incremental backup, current table set is " + backupManager.getIncrementalBackupTableSet()); try { - IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager); - - newTimestamps = incrBackupManager.getIncrBackupLogFileList(conn, backupContext); + newTimestamps = ((IncrementalBackupManager)backupManager). + getIncrBackupLogFileList(conn, backupContext); } catch (Exception e) { // fail the overall backup and return - FullTableBackupClient.failBackup(conn, backupContext, backupManager, e, + failBackup(conn, backupContext, backupManager, e, "Unexpected Exception : ", BackupType.INCREMENTAL, conf); } @@ -198,7 +179,7 @@ public class IncrementalTableBackupClient { } catch (Exception e) { String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId; // fail the overall backup and return - FullTableBackupClient.failBackup(conn, backupContext, backupManager, e, msg, + failBackup(conn, backupContext, backupManager, e, msg, BackupType.INCREMENTAL, conf); } // case INCR_BACKUP_COMPLETE: @@ -223,11 +204,11 @@ public class IncrementalTableBackupClient { .getRSLogTimestampMins(newTableSetTimestampMap)); backupManager.writeBackupStartCode(newStartCode); // backup complete - FullTableBackupClient.completeBackup(conn, backupContext, backupManager, + completeBackup(conn, backupContext, backupManager, BackupType.INCREMENTAL, conf); } catch (IOException e) { - FullTableBackupClient.failBackup(conn, backupContext, backupManager, e, + failBackup(conn, backupContext, backupManager, e, "Unexpected Exception : ", BackupType.INCREMENTAL, conf); } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java index 91f2d68..a69cf96 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; import org.apache.hadoop.hbase.backup.RestoreRequest; -import org.apache.hadoop.hbase.backup.impl.BackupManifest; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -52,10 +51,6 @@ public class RestoreTablesClient { private String targetRootDir; private boolean isOverwrite; - public RestoreTablesClient() { - // Required by the Procedure framework to create the procedure on replay - } - public RestoreTablesClient(Connection conn, RestoreRequest request) throws IOException { this.targetRootDir = request.getBackupRootDir(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java new file mode 100644 index 0000000..ce1ed8f --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java @@ -0,0 +1,386 @@ +/** + * 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.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; + +@InterfaceAudience.Private +public abstract class TableBackupClient { + private static final Log LOG = LogFactory.getLog(TableBackupClient.class); + + protected Configuration conf; + protected Connection conn; + protected String backupId; + protected List tableList; + protected HashMap newTimestamps = null; + + protected BackupManager backupManager; + protected BackupInfo backupContext; + + public TableBackupClient(final Connection conn, final String backupId, + BackupRequest request) + throws IOException { + if (request.getBackupType() == BackupType.FULL) { + backupManager = new BackupManager(conn, conn.getConfiguration()); + } else { + backupManager = new IncrementalBackupManager(conn, conn.getConfiguration()); + } + this.backupId = backupId; + this.tableList = request.getTableList(); + this.conn = conn; + this.conf = conn.getConfiguration(); + backupContext = + backupManager.createBackupContext(backupId, request.getBackupType(), tableList, + request.getTargetRootDir(), + request.getWorkers(), request.getBandwidth()); + if (tableList == null || tableList.isEmpty()) { + this.tableList = new ArrayList<>(backupContext.getTables()); + } + } + + /** + * Begin the overall backup. + * @param backupContext backup context + * @throws IOException exception + */ + protected void beginBackup(BackupManager backupManager, BackupInfo backupContext) throws IOException { + backupManager.setBackupContext(backupContext); + // set the start timestamp of the overall backup + long startTs = EnvironmentEdgeManager.currentTime(); + backupContext.setStartTs(startTs); + // set overall backup status: ongoing + backupContext.setState(BackupState.RUNNING); + LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + "."); + + backupManager.updateBackupInfo(backupContext); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup session " + backupContext.getBackupId() + " has been started."); + } + } + + private String getMessage(Exception e) { + String msg = e.getMessage(); + if (msg == null || msg.equals("")) { + msg = e.getClass().getName(); + } + return msg; + } + + /** + * Delete HBase snapshot for backup. + * @param backupCtx backup context + * @throws Exception exception + */ + private void + deleteSnapshot(final Connection conn, BackupInfo backupCtx, Configuration conf) + throws IOException { + LOG.debug("Trying to delete snapshot for full backup."); + for (String snapshotName : backupCtx.getSnapshotNames()) { + if (snapshotName == null) { + continue; + } + LOG.debug("Trying to delete snapshot: " + snapshotName); + + try (Admin admin = conn.getAdmin();) { + admin.deleteSnapshot(snapshotName); + } catch (IOException ioe) { + LOG.debug("when deleting snapshot " + snapshotName, ioe); + } + LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupCtx.getBackupId() + + " succeeded."); + } + } + + /** + * Clean up directories with prefix "exportSnapshot-", which are generated when exporting + * snapshots. + * @throws IOException exception + */ + private void cleanupExportSnapshotLog(Configuration conf) throws IOException { + FileSystem fs = FSUtils.getCurrentFileSystem(conf); + Path stagingDir = + new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory() + .toString())); + FileStatus[] files = FSUtils.listStatus(fs, stagingDir); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("exportSnapshot-")) { + LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); + if (FSUtils.delete(fs, file.getPath(), true) == false) { + LOG.warn("Can not delete " + file.getPath()); + } + } + } + } + + /** + * Clean up the uncompleted data at target directory if the ongoing backup has already entered the + * copy phase. + */ + private void cleanupTargetDir(BackupInfo backupContext, Configuration conf) { + try { + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + LOG.debug("Trying to cleanup up target dir. Current backup phase: " + + backupContext.getPhase()); + if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY) + || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY) + || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) { + FileSystem outputFs = + FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf); + + // now treat one backup as a transaction, clean up data that has been partially copied at + // table level + for (TableName table : backupContext.getTables()) { + Path targetDirPath = + new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(), + backupContext.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString() + + " done."); + } else { + LOG.info("No data has been copied to " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir); + if (backups == null || backups.length == 0) { + outputFs.delete(tableDir, true); + LOG.debug(tableDir.toString() + " is empty, remove it."); + } + } + } + + } catch (IOException e1) { + LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at " + + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Fail the overall backup. + * @param backupContext backup context + * @param e exception + * @throws Exception exception + */ + protected void failBackup(Connection conn, BackupInfo backupContext, BackupManager backupManager, + Exception e, String msg, BackupType type, Configuration conf) throws IOException { + LOG.error(msg + getMessage(e), e); + // If this is a cancel exception, then we've already cleaned. + + // set the failure timestamp of the overall backup + backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); + + // set failure message + backupContext.setFailedMsg(e.getMessage()); + + // set overall backup status: failed + backupContext.setState(BackupState.FAILED); + + // compose the backup failed data + String backupFailedData = + "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs() + + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase() + + ",failedmessage=" + backupContext.getFailedMsg(); + LOG.error(backupFailedData); + + backupManager.updateBackupInfo(backupContext); + + // if full backup, then delete HBase snapshots if there already are snapshots taken + // and also clean up export snapshot log files if exist + if (type == BackupType.FULL) { + deleteSnapshot(conn, backupContext, conf); + cleanupExportSnapshotLog(conf); + } + + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + // For incremental backup, DistCp logs will be cleaned with the targetDir. + cleanupTargetDir(backupContext, conf); + LOG.info("Backup " + backupContext.getBackupId() + " failed."); + } + + + /** + * Add manifest for the current backup. The manifest is stored within the table backup directory. + * @param backupContext The current backup context + * @throws IOException exception + * @throws BackupException exception + */ + private void addManifest(BackupInfo backupContext, BackupManager backupManager, + BackupType type, Configuration conf) throws IOException, BackupException { + // set the overall backup phase : store manifest + backupContext.setPhase(BackupPhase.STORE_MANIFEST); + + BackupManifest manifest; + + // Since we have each table's backup in its own directory structure, + // we'll store its manifest with the table directory. + for (TableName table : backupContext.getTables()) { + manifest = new BackupManifest(backupContext, table); + ArrayList ancestors = backupManager.getAncestors(backupContext, table); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + + if (type == BackupType.INCREMENTAL) { + // We'll store the log timestamps for this table only in its manifest. + HashMap> tableTimestampMap = + new HashMap>(); + tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table)); + manifest.setIncrTimestampMap(tableTimestampMap); + ArrayList ancestorss = backupManager.getAncestors(backupContext); + for (BackupImage image : ancestorss) { + manifest.addDependentImage(image); + } + } + manifest.store(conf); + } + + // For incremental backup, we store a overall manifest in + // /WALs/ + // This is used when created the next incremental backup + if (type == BackupType.INCREMENTAL) { + manifest = new BackupManifest(backupContext); + // set the table region server start and end timestamps for incremental backup + manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap()); + ArrayList ancestors = backupManager.getAncestors(backupContext); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + manifest.store(conf); + } + } + + /** + * Get backup request meta data dir as string. + * @param backupContext backup context + * @return meta data dir + */ + private String obtainBackupMetaDataStr(BackupInfo backupContext) { + StringBuffer sb = new StringBuffer(); + sb.append("type=" + backupContext.getType() + ",tablelist="); + for (TableName table : backupContext.getTables()) { + sb.append(table + ";"); + } + if (sb.lastIndexOf(";") > 0) { + sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); + } + sb.append(",targetRootDir=" + backupContext.getTargetRootDir()); + + return sb.toString(); + } + + /** + * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying + * hlogs. + * @throws IOException exception + */ + private void cleanupDistCpLog(BackupInfo backupContext, Configuration conf) + throws IOException { + Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = FSUtils.listStatus(fs, rootPath); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("_distcp_logs")) { + LOG.debug("Delete log files of DistCp: " + file.getPath().getName()); + FSUtils.delete(fs, file.getPath(), true); + } + } + } + + /** + * Complete the overall backup. + * @param backupContext backup context + * @throws Exception exception + */ + protected void completeBackup(final Connection conn, BackupInfo backupContext, + BackupManager backupManager, BackupType type, Configuration conf) throws IOException { + // set the complete timestamp of the overall backup + backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); + // set overall backup status: complete + backupContext.setState(BackupState.COMPLETE); + backupContext.setProgress(100); + // add and store the manifest for the backup + addManifest(backupContext, backupManager, type, conf); + + // after major steps done and manifest persisted, do convert if needed for incremental backup + /* in-fly convert code here, provided by future jira */ + LOG.debug("in-fly convert code here, provided by future jira"); + + // compose the backup complete data + String backupCompleteData = + obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs() + + ",completets=" + backupContext.getEndTs() + ",bytescopied=" + + backupContext.getTotalBytesCopied(); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData); + } + backupManager.updateBackupInfo(backupContext); + + // when full backup is done: + // - delete HBase snapshot + // - clean up directories with prefix "exportSnapshot-", which are generated when exporting + // snapshots + if (type == BackupType.FULL) { + deleteSnapshot(conn, backupContext, conf); + cleanupExportSnapshotLog(conf); + } else if (type == BackupType.INCREMENTAL) { + cleanupDistCpLog(backupContext, conf); + } + LOG.info("Backup " + backupContext.getBackupId() + " completed."); + } + + /** + * Backup request execution + * @throws IOException + */ + public abstract void execute() throws IOException; + + +} + diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java index cdde89e..8d404d9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java @@ -48,12 +48,9 @@ import org.apache.hadoop.tools.DistCpOptions; import org.apache.zookeeper.KeeperException.NoNodeException; /** * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot, - * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper - * implementation. The other is copying for incremental log files, which bases on extending - * DistCp's function with copy progress reporting to ZooKeeper implementation. + * which bases on extending ExportSnapshot's function. The other is copying for incremental + * log files, which bases on extending DistCp's function. * - * For now this is only a wrapper. The other features such as progress and increment backup will be - * implemented in future jira */ @InterfaceAudience.Private @@ -122,7 +119,7 @@ public class MapReduceBackupCopyTask implements BackupCopyTask { /** * Update the ongoing backup with new progress. * @param backupContext backup context - * + * * @param newProgress progress * @param bytesCopied bytes copied * @throws NoNodeException exception @@ -199,7 +196,7 @@ public class MapReduceBackupCopyTask implements BackupCopyTask { // Get the total length of the source files List srcs = ((DistCpOptions) fieldInputOptions.get(this)).getSourcePaths(); - + long totalSrcLgth = 0; for (Path aSrc : srcs) { totalSrcLgth += BackupServerUtil.getFilesLength(aSrc.getFileSystem(getConf()), aSrc); @@ -277,7 +274,7 @@ public class MapReduceBackupCopyTask implements BackupCopyTask { } - + /** * Do backup copy based on different types. * @param context The backup context @@ -327,7 +324,7 @@ public class MapReduceBackupCopyTask implements BackupCopyTask { @Override public void cancelCopyJob(String jobId) throws IOException { - JobID id = JobID.forName(jobId); + JobID id = JobID.forName(jobId); Cluster cluster = new Cluster(getConf()); try { Job job = cluster.getJob(id); @@ -338,7 +335,7 @@ public class MapReduceBackupCopyTask implements BackupCopyTask { if (job.isComplete() || job.isRetired()) { return; } - + job.killJob(); LOG.debug("Killed job " + id); } catch (InterruptedException e) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java index 46e2faf..88e1050 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java @@ -38,10 +38,17 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManager; import org.apache.hadoop.hbase.procedure.Procedure; import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.zookeeper.KeeperException; +/** + * Master procedure manager for coordinated cluster-wide + * WAL roll operation, which is run during backup operation, + * see {@link MasterProcedureManager} and and {@link RegionServerProcedureManager} + * + */ public class LogRollMasterProcedureManager extends MasterProcedureManager { public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc"; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java index ed5c44e..b69291c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java @@ -86,9 +86,9 @@ public class LogRollBackupSubprocedure extends Subprocedure { LOG.info("Trying to roll log in backup subprocedure, " + " highest log num: " + highest + " on " + rss.getServerName()); - ((HRegionServer)rss).walRoller.requestRollAll(); + ((HRegionServer)rss).getWalRoller().requestRollAll(); long start = EnvironmentEdgeManager.currentTime(); - while (!((HRegionServer)rss).walRoller.walRollFinished()) { + while (!((HRegionServer)rss).getWalRoller().walRollFinished()) { Thread.sleep(20); } LOG.debug("log roll took " + (EnvironmentEdgeManager.currentTime()-start)); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java index 5887d21..a550725 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java @@ -36,12 +36,11 @@ import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs; import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.SubprocedureFactory; -import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.zookeeper.KeeperException; /** - * This manager class handles the work dealing with backup for a {@link HRegionServer}. + * This manager class handles the work dealing with distributed WAL roll request. *

* This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is * responsible by this region server. If any failures occur with the subprocedure, the manager's diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java index 9d522c6..1d0642b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java @@ -72,7 +72,7 @@ public class RestoreServerUtil { public static final Log LOG = LogFactory.getLog(RestoreServerUtil.class); - private final String[] ignoreDirs = { "recovered.edits" }; + private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR }; private final long TABLE_AVAILABILITY_WAIT_TIME = 180000; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java index 3865ba9..b85b839 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java @@ -51,10 +51,9 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { * @param procedureClass procedure type name is a category for when there are multiple kinds of * procedures.-- this becomes a znode so be aware of the naming restrictions * @param coordName name of the node running the coordinator - * @throws KeeperException if an unexpected zk error occurs */ public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher, - String procedureClass, String coordName) throws IOException { + String procedureClass, String coordName) { this.watcher = watcher; this.procedureType = procedureClass; this.coordName = coordName; @@ -181,6 +180,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { * Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about. * @return true if succeed, false if encountered initialization errors. */ + @Override final public boolean start(final ProcedureCoordinator coordinator) { if (this.coordinator != null) { throw new IllegalStateException( diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index eb0c9c0..a1c3c91 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -183,6 +183,9 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.data.Stat; +import sun.misc.Signal; +import sun.misc.SignalHandler; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; @@ -194,9 +197,6 @@ import com.google.protobuf.RpcController; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; -import sun.misc.Signal; -import sun.misc.SignalHandler; - /** * HRegionServer makes a set of HRegions available to clients. It checks in with * the HMaster. There are many HRegionServers in a single HBase deployment. @@ -368,7 +368,7 @@ public class HRegionServer extends HasThread implements // WAL roller. log is protected rather than private to avoid // eclipse warning when accessed by inner classes - public final LogRoller walRoller; + protected final LogRoller walRoller; // Lazily initialized if this RegionServer hosts a meta table. final AtomicReference metawalRoller = new AtomicReference(); @@ -621,6 +621,7 @@ public class HRegionServer extends HasThread implements if (!SystemUtils.IS_OS_WINDOWS) { Signal.handle(new Signal("HUP"), new SignalHandler() { + @Override public void handle(Signal signal) { getConfiguration().reloadConfiguration(); configurationManager.notifyAllObservers(getConfiguration()); @@ -635,7 +636,7 @@ public class HRegionServer extends HasThread implements int cleanerInterval = conf.getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000); this.compactedFileDischarger = - new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this); + new CompactedHFilesDischarger(cleanerInterval, this, this); choreService.scheduleChore(compactedFileDischarger); } @@ -1897,6 +1898,10 @@ public class HRegionServer extends HasThread implements return wal; } + public LogRoller getWalRoller() { + return walRoller; + } + @Override public Connection getConnection() { return getClusterConnection(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 3eeac05..75fee47 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -173,7 +173,7 @@ public class TestBackupBase { String backupId; try { conn = ConnectionFactory.createConnection(conf1); - badmin = new HBaseBackupAdmin(conn); + badmin = new BackupAdminImpl(conn); BackupRequest request = new BackupRequest(); request.setBackupType(type).setTableList(tables).setTargetRootDir(path); backupId = badmin.backupTables(request); @@ -271,7 +271,7 @@ public class TestBackupBase { } protected BackupAdmin getBackupAdmin() throws IOException { - return new HBaseBackupAdmin(TEST_UTIL.getConnection()); + return new BackupAdminImpl(TEST_UTIL.getConnection()); } /** diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java index 5586112..af9691a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java @@ -32,6 +32,19 @@ import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestBackupCommandLineTool { + + private final static String USAGE_DESCRIBE = "Usage: bin/hbase backup describe "; + private final static String USAGE_CREATE = "Usage: bin/hbase backup create"; + private final static String USAGE_HISTORY = "Usage: bin/hbase backup history"; + private final static String USAGE_BACKUP = "Usage: bin/hbase backup"; + private final static String USAGE_DELETE = "Usage: bin/hbase backup delete"; + private final static String USAGE_PROGRESS = "Usage: bin/hbase backup progress"; + private final static String USAGE_SET = "Usage: bin/hbase backup set"; + private final static String USAGE_RESTORE = "Usage: bin/hbase restore"; + + + + Configuration conf; @Before public void setUpBefore() throws Exception { @@ -48,7 +61,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup describe ") >= 0); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -57,7 +70,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup describe ") >= 0); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -66,7 +79,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup describe ") >= 0); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); } @Test @@ -78,7 +91,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -87,7 +100,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -96,7 +109,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); } @Test @@ -108,7 +121,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup history") >= 0); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -117,7 +130,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup history") >= 0); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); } @@ -130,7 +143,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup delete") >= 0); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -139,7 +152,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup delete") >= 0); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -148,7 +161,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup delete") >= 0); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); } @Test @@ -160,7 +173,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup progress") >= 0); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -169,7 +182,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup progress") >= 0); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); } @Test @@ -181,7 +194,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup set") >= 0); + assertTrue(output.indexOf(USAGE_SET) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -190,7 +203,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup set") >= 0); + assertTrue(output.indexOf(USAGE_SET) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -199,7 +212,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup set") >= 0); + assertTrue(output.indexOf(USAGE_SET) >= 0); } @@ -212,7 +225,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -221,7 +234,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); } @Test @@ -233,7 +246,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase restore") >= 0); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -242,7 +255,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase restore") >= 0); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); } @Test @@ -254,7 +267,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -263,7 +276,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); } @@ -277,7 +290,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -286,7 +299,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -295,7 +308,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -304,7 +317,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -313,7 +326,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup") >= 0); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); } @Test @@ -325,7 +338,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase restore") >= 0); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); } @@ -338,7 +351,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -347,7 +360,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); baos = new ByteArrayOutputStream(); System.setOut(new PrintStream(baos)); @@ -356,7 +369,7 @@ public class TestBackupCommandLineTool { output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup create") >= 0); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); } @Test @@ -368,7 +381,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup delete") >= 0); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); } @@ -381,7 +394,7 @@ public class TestBackupCommandLineTool { String output = baos.toString(); System.out.println(baos.toString()); - assertTrue(output.indexOf("Usage: bin/hbase backup history") >= 0); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java index 1caba22..f4b9499 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java @@ -28,7 +28,7 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HBaseAdmin; @@ -60,7 +60,7 @@ public class TestBackupMultipleDeletes extends TestBackupBase { HBaseAdmin admin = null; Connection conn = ConnectionFactory.createConnection(conf1); admin = (HBaseAdmin) conn.getAdmin(); - BackupAdmin client = new HBaseBackupAdmin(conn); + BackupAdmin client = new BackupAdminImpl(conn); BackupRequest request = new BackupRequest(); request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR); String backupIdFull = client.backupTables(request); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java index 9a845ba..38872d2 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -84,7 +84,7 @@ public class TestIncrementalBackup extends TestBackupBase { HBaseAdmin admin = null; admin = (HBaseAdmin) conn.getAdmin(); - HBaseBackupAdmin client = new HBaseBackupAdmin(conn); + BackupAdminImpl client = new BackupAdminImpl(conn); BackupRequest request = new BackupRequest(); request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java index 0a73888..cc5dcef 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java @@ -25,7 +25,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.util.RestoreServerUtil; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -63,7 +63,7 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase { HBaseAdmin admin = null; Connection conn = ConnectionFactory.createConnection(conf1); admin = (HBaseAdmin) conn.getAdmin(); - HBaseBackupAdmin client = new HBaseBackupAdmin(conn); + BackupAdminImpl client = new BackupAdminImpl(conn); BackupRequest request = new BackupRequest(); request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);