org.apache.hbase
hbase-hadoop-compat
diff --git hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index e967083..2913dcb 100644
--- hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -400,6 +400,8 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
} else if (tableName.equals(TableName.valueOf("hbase:replication"))) {
description = "The hbase:replication table tracks cross cluster replication through " +
"WAL file offsets.";
+ } else if (tableName.equals(TableName.BACKUP_TABLE_NAME)) {
+ description = "The hbase:backup table stores backup system information.";
}
%java>
| <% description %> |
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..0b8de28
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+/**
+ * The administrative API for HBase Backup. Construct an instance
+ * and call {@link #close()} afterwards.
+ * BackupAdmin can be used to create backups, restore data from backups and for
+ * other backup-related operations.
+ *
+ * @since 2.0
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+
+public interface BackupAdmin extends Closeable{
+
+ /**
+ * Backs up given list of tables fully. Synchronous operation.
+ *
+ * @param userRequest BackupRequest instance
+ * @return the backup Id
+ */
+
+ public String backupTables(final BackupRequest userRequest) throws IOException;
+
+ /**
+ * Backs up given list of tables fully. Asynchronous operation.
+ *
+ * @param userRequest BackupRequest instance
+ * @return the backup Id future
+ */
+ public Future backupTablesAsync(final BackupRequest userRequest) throws IOException;
+
+ /**
+ * Restore backup
+ * @param request - restore request
+ * @throws IOException exception
+ */
+ public void restore(RestoreRequest request) throws IOException;
+
+ /**
+ * Restore backup
+ * @param request - restore request
+ * @return Future which client can wait on
+ * @throws IOException exception
+ */
+ public Future restoreAsync(RestoreRequest request) throws IOException;
+
+ /**
+ * Describe backup image command
+ * @param backupId - backup id
+ * @return backup info
+ * @throws IOException exception
+ */
+ public BackupInfo getBackupInfo(String backupId) throws IOException;
+
+ /**
+ * Show backup progress command
+ * @param backupId - backup id (may be null)
+ * @return backup progress (0-100%), -1 if no active sessions
+ * or session not found
+ * @throws IOException exception
+ */
+ public int getProgress(String backupId) throws IOException;
+
+ /**
+ * Delete backup image command
+ * @param backupIds - backup id
+ * @return total number of deleted sessions
+ * @throws IOException exception
+ */
+ public int deleteBackups(String[] backupIds) throws IOException;
+
+ /**
+ * Show backup history command
+ * @param n - last n backup sessions
+ * @return list of backup infos
+ * @throws IOException exception
+ */
+ public List getHistory(int n) throws IOException;
+
+
+ /**
+ * Show backup history command with filters
+ * @param n - last n backup sessions
+ * @param f - list of filters
+ * @return list of backup infos
+ * @throws IOException exception
+ */
+ public List getHistory(int n, BackupInfo.Filter ... f) throws IOException;
+
+
+ /**
+ * Backup sets list command - list all backup sets. Backup set is
+ * a named group of tables.
+ * @return all registered backup sets
+ * @throws IOException exception
+ */
+ public List listBackupSets() throws IOException;
+
+ /**
+ * Backup set describe command. Shows list of tables in
+ * this particular backup set.
+ * @param name set name
+ * @return backup set description or null
+ * @throws IOException exception
+ */
+ public BackupSet getBackupSet(String name) throws IOException;
+
+ /**
+ * Delete backup set command
+ * @param name - backup set name
+ * @return true, if success, false - otherwise
+ * @throws IOException exception
+ */
+ public boolean deleteBackupSet(String name) throws IOException;
+
+ /**
+ * Add tables to backup set command
+ * @param name - name of backup set.
+ * @param tables - list of tables to be added to this set.
+ * @throws IOException exception
+ */
+ public void addToBackupSet(String name, TableName[] tables) throws IOException;
+
+ /**
+ * Remove tables from backup set
+ * @param name - name of backup set.
+ * @param tables - list of tables to be removed from this set.
+ * @throws IOException exception
+ */
+ public void removeFromBackupSet(String name, String[] tables) throws IOException;
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyTask.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyTask.java
new file mode 100644
index 0000000..26a7e44
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyTask.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BackupCopyTask extends Configurable {
+
+ /**
+ * Copy backup data to destination
+ * @param backupContext context object
+ * @param backupManager backup manager
+ * @param conf configuration
+ * @param backupType backup type (FULL or INCREMENTAL)
+ * @param options array of options (implementation-specific)
+ * @return result (0 - success, -1 failure )
+ * @throws IOException exception
+ */
+ int copy(BackupInfo backupContext, BackupManager backupManager, Configuration conf,
+ BackupType copyType, String[] options) throws IOException;
+
+
+ /**
+ * Cancel copy job
+ * @param jobHandler - copy job handler
+ * @throws IOException
+ */
+ void cancelCopyJob(String jobHandler) throws IOException;
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..07f39b6
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,196 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.cli.CommandLine;
+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.HBaseConfiguration;
+import org.apache.hadoop.hbase.backup.impl.BackupCommands;
+import org.apache.hadoop.hbase.backup.util.LogUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupDriver extends AbstractHBaseTool implements BackupRestoreConstants {
+
+ private static final Log LOG = LogFactory.getLog(BackupDriver.class);
+ private CommandLine cmd;
+
+ public BackupDriver() throws IOException {
+ init();
+ }
+
+ protected void init() throws IOException {
+ // disable irrelevant loggers to avoid it mess up command output
+ LogUtils.disableZkAndClientLoggers(LOG);
+ }
+
+ private int parseAndRun(String[] args) throws IOException {
+
+ String cmd = null;
+ String[] remainArgs = null;
+ if (args == null || args.length == 0) {
+ printToolUsage();
+ return -1;
+ } else {
+ cmd = args[0];
+ remainArgs = new String[args.length - 1];
+ if (args.length > 1) {
+ System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
+ }
+ }
+
+ BackupCommand type = BackupCommand.HELP;
+ if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.CREATE;
+ } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.HELP;
+ } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.DELETE;
+ } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.DESCRIBE;
+ } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.HISTORY;
+ } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.PROGRESS;
+ } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) {
+ type = BackupCommand.SET;
+ } else {
+ System.out.println("Unsupported command for backup: " + cmd);
+ printToolUsage();
+ return -1;
+ }
+
+ // enable debug logging
+ Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+ if (this.cmd.hasOption(OPTION_DEBUG)) {
+ backupClientLogger.setLevel(Level.DEBUG);
+ } else {
+ backupClientLogger.setLevel(Level.INFO);
+ }
+
+ // TODO: get rid of Command altogether?
+ BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd);
+ if (type == BackupCommand.CREATE && conf != null) {
+ ((BackupCommands.CreateCommand) command).setConf(conf);
+ }
+ try {
+ command.execute();
+ } catch (IOException e) {
+ if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) {
+ return -1;
+ }
+ throw e;
+ }
+ return 0;
+ }
+
+ @Override
+ protected void addOptions() {
+ // define supported options
+ addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+ addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC);
+ addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC);
+ addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC);
+ addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC);
+ addOptWithArg(OPTION_SET, OPTION_SET_DESC);
+ addOptWithArg(OPTION_PATH, OPTION_PATH_DESC);
+ }
+
+ @Override
+ protected void processOptions(CommandLine cmd) {
+ this.cmd = cmd;
+ }
+
+ @Override
+ protected int doWork() throws Exception {
+ return parseAndRun(cmd.getArgs());
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration conf = HBaseConfiguration.create();
+ Path hbasedir = FSUtils.getRootDir(conf);
+ URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+ FSUtils.setFsDefault(conf, new Path(defaultFs));
+ int ret = ToolRunner.run(conf, new BackupDriver(), args);
+ System.exit(ret);
+ }
+
+ @Override
+ public int run(String[] args) throws IOException {
+ if (conf == null) {
+ LOG.error("Tool configuration is not initialized");
+ throw new NullPointerException("conf");
+ }
+
+ CommandLine cmd;
+ try {
+ // parse the command line arguments
+ cmd = parseArgs(args);
+ cmdLineArgs = args;
+ } catch (Exception e) {
+ System.out.println("Error when parsing command-line arguments: " + e.getMessage());
+ printToolUsage();
+ return EXIT_FAILURE;
+ }
+
+ if (!sanityCheckOptions(cmd)) {
+ printToolUsage();
+ return EXIT_FAILURE;
+ }
+
+ processOptions(cmd);
+
+ int ret = EXIT_FAILURE;
+ try {
+ ret = doWork();
+ } catch (Exception e) {
+ LOG.error("Error running command-line tool", e);
+ return EXIT_FAILURE;
+ }
+ return ret;
+ }
+
+ @Override
+ protected boolean sanityCheckOptions(CommandLine cmd) {
+ boolean success = true;
+ for (String reqOpt : requiredOptions) {
+ if (!cmd.hasOption(reqOpt)) {
+ System.out.println("Required option -" + reqOpt + " is missing");
+ success = false;
+ }
+ }
+ return success;
+ }
+
+ protected void printToolUsage() throws IOException {
+ System.out.println(BackupCommands.USAGE);
+ }
+}
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
new file mode 100644
index 0000000..4ea0299
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,562 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableBackupStatus;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupInfo implements Comparable {
+ private static final Log LOG = LogFactory.getLog(BackupInfo.class);
+
+ public static interface Filter {
+
+ /**
+ * Filter interface
+ * @param info backup info
+ * @return true if info passes filter, false otherwise
+ */
+ public boolean apply(BackupInfo info);
+ }
+
+ /**
+ * Backup status flag
+ */
+ public static enum BackupState {
+ WAITING, RUNNING, COMPLETE, FAILED, ANY;
+ }
+
+ /**
+ * Backup phase
+ */
+ public static enum BackupPhase {
+ SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+ }
+
+ /**
+ * Backup id
+ */
+ private String backupId;
+
+ /**
+ * Backup type, full or incremental
+ */
+ private BackupType type;
+
+ /**
+ * Target root directory for storing the backup files
+ */
+ private String targetRootDir;
+
+ /**
+ * Backup state
+ */
+ private BackupState state;
+
+ /**
+ * Backup phase
+ */
+ private BackupPhase phase;
+
+ /**
+ * Backup failure message
+ */
+ private String failedMsg;
+
+ /**
+ * Backup status map for all tables
+ */
+ private Map backupStatusMap;
+
+ /**
+ * Actual start timestamp of a backup process
+ */
+ private long startTs;
+
+ /**
+ * Actual end timestamp of the backup process
+ */
+ private long endTs;
+
+ /**
+ * Total bytes of incremental logs copied
+ */
+ private long totalBytesCopied;
+
+ /**
+ * For incremental backup, a location of a backed-up hlogs
+ */
+ private String hlogTargetDir = null;
+
+ /**
+ * Incremental backup file list
+ */
+ transient private List incrBackupFileList;
+
+ /**
+ * New region server log timestamps for table set after distributed log roll
+ * key - table name, value - map of RegionServer hostname -> last log rolled timestamp
+ */
+ transient private HashMap> tableSetTimestampMap;
+
+ /**
+ * Backup progress in %% (0-100)
+ */
+ private int progress;
+
+ /**
+ * Distributed job id
+ */
+ private String jobId;
+
+ /**
+ * Number of parallel workers. -1 - system defined
+ */
+ private int workers = -1;
+
+ /**
+ * Bandwidth per worker in MB per sec. -1 - unlimited
+ */
+ private long bandwidth = -1;
+
+ public BackupInfo() {
+ backupStatusMap = new HashMap();
+ }
+
+ public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
+ this();
+ this.backupId = backupId;
+ this.type = type;
+ this.targetRootDir = targetRootDir;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("CreateBackupContext: " + tables.length + " " + tables[0]);
+ }
+ this.addTables(tables);
+
+ if (type == BackupType.INCREMENTAL) {
+ setHlogTargetDir(BackupClientUtil.getLogBackupDir(targetRootDir, backupId));
+ }
+
+ this.startTs = 0;
+ this.endTs = 0;
+ }
+
+ public String getJobId() {
+ return jobId;
+ }
+
+ public void setJobId(String jobId) {
+ this.jobId = jobId;
+ }
+
+ public int getWorkers() {
+ return workers;
+ }
+
+ public void setWorkers(int workers) {
+ this.workers = workers;
+ }
+
+ public long getBandwidth() {
+ return bandwidth;
+ }
+
+ public void setBandwidth(long bandwidth) {
+ this.bandwidth = bandwidth;
+ }
+
+ public void setBackupStatusMap(Map backupStatusMap) {
+ this.backupStatusMap = backupStatusMap;
+ }
+
+ public HashMap> getTableSetTimestampMap() {
+ return tableSetTimestampMap;
+ }
+
+ public void
+ setTableSetTimestampMap(HashMap> tableSetTimestampMap) {
+ this.tableSetTimestampMap = tableSetTimestampMap;
+ }
+
+ public String getHlogTargetDir() {
+ return hlogTargetDir;
+ }
+
+ public void setType(BackupType type) {
+ this.type = type;
+ }
+
+ public void setTargetRootDir(String targetRootDir) {
+ this.targetRootDir = targetRootDir;
+ }
+
+ public void setTotalBytesCopied(long totalBytesCopied) {
+ this.totalBytesCopied = totalBytesCopied;
+ }
+
+ /**
+ * Set progress (0-100%)
+ * @param p progress value
+ */
+
+ public void setProgress(int p) {
+ this.progress = p;
+ }
+
+ /**
+ * Get current progress
+ */
+ public int getProgress() {
+ return progress;
+ }
+
+ public String getBackupId() {
+ return backupId;
+ }
+
+ public void setBackupId(String backupId) {
+ this.backupId = backupId;
+ }
+
+ public BackupStatus getBackupStatus(TableName table) {
+ return this.backupStatusMap.get(table);
+ }
+
+ public String getFailedMsg() {
+ return failedMsg;
+ }
+
+ public void setFailedMsg(String failedMsg) {
+ this.failedMsg = failedMsg;
+ }
+
+ public long getStartTs() {
+ return startTs;
+ }
+
+ public void setStartTs(long startTs) {
+ this.startTs = startTs;
+ }
+
+ public long getEndTs() {
+ return endTs;
+ }
+
+ public void setEndTs(long endTs) {
+ this.endTs = endTs;
+ }
+
+ public long getTotalBytesCopied() {
+ return totalBytesCopied;
+ }
+
+ public BackupState getState() {
+ return state;
+ }
+
+ public void setState(BackupState flag) {
+ this.state = flag;
+ }
+
+ public BackupPhase getPhase() {
+ return phase;
+ }
+
+ public void setPhase(BackupPhase phase) {
+ this.phase = phase;
+ }
+
+ public BackupType getType() {
+ return type;
+ }
+
+ public void setSnapshotName(TableName table, String snapshotName) {
+ this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+ }
+
+ public String getSnapshotName(TableName table) {
+ return this.backupStatusMap.get(table).getSnapshotName();
+ }
+
+ public List getSnapshotNames() {
+ List snapshotNames = new ArrayList();
+ for (BackupStatus backupStatus : this.backupStatusMap.values()) {
+ snapshotNames.add(backupStatus.getSnapshotName());
+ }
+ return snapshotNames;
+ }
+
+ public Set getTables() {
+ return this.backupStatusMap.keySet();
+ }
+
+ public List getTableNames() {
+ return new ArrayList(backupStatusMap.keySet());
+ }
+
+ public void addTables(TableName[] tables) {
+ for (TableName table : tables) {
+ BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+ this.backupStatusMap.put(table, backupStatus);
+ }
+ }
+
+ public void setTables(List tables) {
+ this.backupStatusMap.clear();
+ for (TableName table : tables) {
+ BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+ this.backupStatusMap.put(table, backupStatus);
+ }
+ }
+
+ public String getTargetRootDir() {
+ return targetRootDir;
+ }
+
+ public void setHlogTargetDir(String hlogTagetDir) {
+ this.hlogTargetDir = hlogTagetDir;
+ }
+
+ public String getHLogTargetDir() {
+ return hlogTargetDir;
+ }
+
+ public List getIncrBackupFileList() {
+ return incrBackupFileList;
+ }
+
+ public void setIncrBackupFileList(List incrBackupFileList) {
+ this.incrBackupFileList = incrBackupFileList;
+ }
+
+ /**
+ * Set the new region server log timestamps after distributed log roll
+ * @param newTableSetTimestampMap table timestamp map
+ */
+ public void
+ setIncrTimestampMap(HashMap> newTableSetTimestampMap) {
+ this.tableSetTimestampMap = newTableSetTimestampMap;
+ }
+
+ /**
+ * Get new region server log timestamps after distributed log roll
+ * @return new region server log timestamps
+ */
+ public HashMap> getIncrTimestampMap() {
+ return this.tableSetTimestampMap;
+ }
+
+ public TableName getTableBySnapshot(String snapshotName) {
+ for (Entry entry : this.backupStatusMap.entrySet()) {
+ if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+ return entry.getKey();
+ }
+ }
+ return null;
+ }
+
+ public BackupProtos.BackupInfo toProtosBackupInfo() {
+ BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder();
+ builder.setBackupId(getBackupId());
+ setBackupStatusMap(builder);
+ builder.setEndTs(getEndTs());
+ if (getFailedMsg() != null) {
+ builder.setFailedMessage(getFailedMsg());
+ }
+ if (getState() != null) {
+ builder.setState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
+ }
+ if (getPhase() != null) {
+ builder.setPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
+ }
+
+ builder.setProgress(getProgress());
+ builder.setStartTs(getStartTs());
+ builder.setTargetRootDir(getTargetRootDir());
+ builder.setType(BackupProtos.BackupType.valueOf(getType().name()));
+ builder.setWorkersNumber(workers);
+ builder.setBandwidth(bandwidth);
+ if (jobId != null) {
+ builder.setJobId(jobId);
+ }
+ return builder.build();
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 33 * type.hashCode() + backupId != null ? backupId.hashCode() : 0;
+ if (targetRootDir != null) {
+ hash = 33 * hash + targetRootDir.hashCode();
+ }
+ hash = 33 * hash + state.hashCode();
+ hash = 33 * hash + phase.hashCode();
+ hash = 33 * hash + (int)(startTs ^ (startTs >>> 32));
+ hash = 33 * hash + (int)(endTs ^ (endTs >>> 32));
+ hash = 33 * hash + (int)(totalBytesCopied ^ (totalBytesCopied >>> 32));
+ if (hlogTargetDir != null) {
+ hash = 33 * hash + hlogTargetDir.hashCode();
+ }
+ if (jobId != null) {
+ hash = 33 * hash + jobId.hashCode();
+ }
+ return hash;
+ }
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof BackupInfo) {
+ BackupInfo other = (BackupInfo) obj;
+ try {
+ return Bytes.equals(toByteArray(), other.toByteArray());
+ } catch (IOException e) {
+ LOG.error(e);
+ return false;
+ }
+ } else {
+ return false;
+ }
+ }
+
+ public byte[] toByteArray() throws IOException {
+ return toProtosBackupInfo().toByteArray();
+ }
+
+ private void setBackupStatusMap(Builder builder) {
+ for (Entry entry : backupStatusMap.entrySet()) {
+ builder.addTableBackupStatus(entry.getValue().toProto());
+ }
+ }
+
+ public static BackupInfo fromByteArray(byte[] data) throws IOException {
+ return fromProto(BackupProtos.BackupInfo.parseFrom(data));
+ }
+
+ public static BackupInfo fromStream(final InputStream stream) throws IOException {
+ return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
+ }
+
+ public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
+ BackupInfo context = new BackupInfo();
+ context.setBackupId(proto.getBackupId());
+ context.setBackupStatusMap(toMap(proto.getTableBackupStatusList()));
+ context.setEndTs(proto.getEndTs());
+ if (proto.hasFailedMessage()) {
+ context.setFailedMsg(proto.getFailedMessage());
+ }
+ if (proto.hasState()) {
+ context.setState(BackupInfo.BackupState.valueOf(proto.getState().name()));
+ }
+
+ context.setHlogTargetDir(BackupClientUtil.getLogBackupDir(proto.getTargetRootDir(),
+ proto.getBackupId()));
+
+ if (proto.hasPhase()) {
+ context.setPhase(BackupPhase.valueOf(proto.getPhase().name()));
+ }
+ if (proto.hasProgress()) {
+ context.setProgress(proto.getProgress());
+ }
+ context.setStartTs(proto.getStartTs());
+ context.setTargetRootDir(proto.getTargetRootDir());
+ context.setType(BackupType.valueOf(proto.getType().name()));
+ context.setWorkers(proto.getWorkersNumber());
+ context.setBandwidth(proto.getBandwidth());
+ if (proto.hasJobId()) {
+ context.setJobId(proto.getJobId());
+ }
+ return context;
+ }
+
+ private static Map toMap(List list) {
+ HashMap map = new HashMap<>();
+ for (TableBackupStatus tbs : list) {
+ map.put(ProtobufUtil.toTableName(tbs.getTable()), BackupStatus.convert(tbs));
+ }
+ return map;
+ }
+
+ public String getShortDescription() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("ID : " + backupId).append("\n");
+ sb.append("Type : " + getType()).append("\n");
+ sb.append("Tables : " + getTableListAsString()).append("\n");
+ sb.append("State : " + getState()).append("\n");
+ Date date = null;
+ Calendar cal = Calendar.getInstance();
+ cal.setTimeInMillis(getStartTs());
+ date = cal.getTime();
+ sb.append("Start time : " + date).append("\n");
+ if (state == BackupState.FAILED) {
+ sb.append("Failed message : " + getFailedMsg()).append("\n");
+ } else if (state == BackupState.RUNNING) {
+ sb.append("Phase : " + getPhase()).append("\n");
+ } else if (state == BackupState.COMPLETE) {
+ cal = Calendar.getInstance();
+ cal.setTimeInMillis(getEndTs());
+ date = cal.getTime();
+ sb.append("End time : " + date).append("\n");
+ }
+ sb.append("Progress : " + getProgress()).append("\n");
+ return sb.toString();
+ }
+
+ public String getStatusAndProgressAsString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("id: ").append(getBackupId()).append(" state: ").append(getState())
+ .append(" progress: ").append(getProgress());
+ return sb.toString();
+ }
+
+ public String getTableListAsString() {
+ return StringUtils.join(backupStatusMap.keySet(), ",");
+ }
+
+ @Override
+ public int compareTo(BackupInfo o) {
+ Long thisTS = Long.valueOf(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
+ Long otherTS = Long.valueOf(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1));
+ return thisTS.compareTo(otherTS);
+ }
+
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..ba8cf33
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO class for backup request
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupRequest {
+ private BackupType type;
+ private List tableList;
+ private String targetRootDir;
+ private int workers = -1;
+ private long bandwidth = -1L;
+ private String backupSetName;
+
+ public BackupRequest() {
+ }
+
+ public BackupRequest setBackupType(BackupType type) {
+ this.type = type;
+ return this;
+ }
+ public BackupType getBackupType() {
+ return this.type;
+ }
+
+ public BackupRequest setTableList(List tableList) {
+ this.tableList = tableList;
+ return this;
+ }
+ public List getTableList() {
+ return this.tableList;
+ }
+
+ public BackupRequest setTargetRootDir(String targetRootDir) {
+ this.targetRootDir = targetRootDir;
+ return this;
+ }
+ public String getTargetRootDir() {
+ return this.targetRootDir;
+ }
+
+ public BackupRequest setWorkers(int workers) {
+ this.workers = workers;
+ return this;
+ }
+ public int getWorkers() {
+ return this.workers;
+ }
+
+ public BackupRequest setBandwidth(long bandwidth) {
+ this.bandwidth = bandwidth;
+ return this;
+ }
+ public long getBandwidth() {
+ return this.bandwidth;
+ }
+
+ public String getBackupSetName() {
+ return backupSetName;
+ }
+
+ public void setBackupSetName(String backupSetName) {
+ this.backupSetName = backupSetName;
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
new file mode 100644
index 0000000..d4bfe69
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public interface BackupRestoreConstants {
+ /**
+ * Backup/Restore constants
+ */
+ public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+ public final static int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER;
+ public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+ public final static boolean BACKUP_ENABLE_DEFAULT = true;
+ // Drivers option list
+ public static final String OPTION_OVERWRITE = "o";
+ public static final String OPTION_OVERWRITE_DESC =
+ "Overwrite data if any of the restore target tables exists";
+
+ public static final String OPTION_CHECK = "c";
+ public static final String OPTION_CHECK_DESC =
+ "Check restore sequence and dependencies only (does not execute the command)";
+
+ public static final String OPTION_SET = "s";
+ public static final String OPTION_SET_DESC = "Backup set name";
+ public static final String OPTION_SET_RESTORE_DESC =
+ "Backup set to restore, mutually exclusive with table list ";
+
+ public static final String OPTION_DEBUG = "d";
+ public static final String OPTION_DEBUG_DESC = "Enable debug loggings";
+
+ public static final String OPTION_TABLE = "t";
+ public static final String OPTION_TABLE_DESC = "Table name. If specified, only backup images,"
+ + " which contain this table will be listed.";
+
+ public static final String OPTION_BANDWIDTH = "b";
+ public static final String OPTION_BANDWIDTH_DESC = "Bandwidth per task (MapReduce task) in MB/s";
+
+ public static final String OPTION_WORKERS = "w";
+ public static final String OPTION_WORKERS_DESC = "Number of parallel MapReduce tasks to execute";
+
+ public static final String OPTION_RECORD_NUMBER = "n";
+ public static final String OPTION_RECORD_NUMBER_DESC =
+ "Number of records of backup history. Default: 10";
+
+ public static final String OPTION_PATH = "p";
+ public static final String OPTION_PATH_DESC = "Backup destination root directory path";
+
+ public static final String OPTION_TABLE_MAPPING = "m";
+ public static final String OPTION_TABLE_MAPPING_DESC =
+ "A comma separated list of target tables. "
+ + "If specified, each table in must have a mapping";
+
+ // delimiter in tablename list in restore command
+ public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+ public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+ public static final String BACKUPID_PREFIX = "backup_";
+
+ public static enum BackupCommand {
+ CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS,
+ SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
+ }
+
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServerFactory.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServerFactory.java
new file mode 100644
index 0000000..ac5bd9b
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServerFactory.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyTask;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreTask;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.util.ReflectionUtils;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupRestoreServerFactory {
+
+ public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class";
+ public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class";
+
+ private BackupRestoreServerFactory(){
+ throw new AssertionError("Instantiating utility class...");
+ }
+
+ /**
+ * Gets backup restore task
+ * @param conf - configuration
+ * @return backup restore task instance
+ */
+ public static RestoreTask getRestoreTask(Configuration conf) {
+ Class extends RestoreTask> cls =
+ conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreTask.class,
+ RestoreTask.class);
+ RestoreTask service = ReflectionUtils.newInstance(cls, conf);
+ service.setConf(conf);
+ return service;
+ }
+
+ /**
+ * Gets backup copy task
+ * @param conf - configuration
+ * @return backup copy task
+ */
+ public static BackupCopyTask getBackupCopyTask(Configuration conf) {
+ Class extends BackupCopyTask> cls =
+ conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyTask.class,
+ BackupCopyTask.class);
+ BackupCopyTask service = ReflectionUtils.newInstance(cls, conf);;
+ service.setConf(conf);
+ return service;
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
new file mode 100644
index 0000000..fd856ec
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+
+/**
+ * Backup status and related information encapsulated for a table.
+ * At this moment only TargetDir and SnapshotName is encapsulated here.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupStatus implements Serializable {
+
+ private static final long serialVersionUID = -5968397963548535982L;
+
+ // table name for backup
+ private transient TableName table;
+
+ // target directory of the backup image for this table
+ private String targetDir;
+
+ // snapshot name for offline/online snapshot
+ private String snapshotName = null;
+
+ public BackupStatus() {
+
+ }
+
+ public BackupStatus(TableName table, String targetRootDir, String backupId) {
+ this.table = table;
+ this.targetDir = BackupClientUtil.getTableBackupDir(targetRootDir, backupId, table);
+ }
+
+ public String getSnapshotName() {
+ return snapshotName;
+ }
+
+ public void setSnapshotName(String snapshotName) {
+ this.snapshotName = snapshotName;
+ }
+
+ public String getTargetDir() {
+ return targetDir;
+ }
+
+ public TableName getTable() {
+ return table;
+ }
+
+ public void setTable(TableName table) {
+ this.table = table;
+ }
+
+ public void setTargetDir(String targetDir) {
+ this.targetDir = targetDir;
+ }
+
+ public static BackupStatus convert(BackupProtos.TableBackupStatus proto)
+ {
+ BackupStatus bs = new BackupStatus();
+ bs.setTable(ProtobufUtil.toTableName(proto.getTable()));
+ bs.setTargetDir(proto.getTargetDir());
+ if(proto.hasSnapshot()){
+ bs.setSnapshotName(proto.getSnapshot());
+ }
+ return bs;
+ }
+
+ public BackupProtos.TableBackupStatus toProto() {
+ BackupProtos.TableBackupStatus.Builder builder =
+ BackupProtos.TableBackupStatus.newBuilder();
+ if(snapshotName != null) {
+ builder.setSnapshot(snapshotName);
+ }
+ builder.setTable(ProtobufUtil.toProtoTableNameShaded(table));
+ builder.setTargetDir(targetDir);
+ return builder.build();
+ }
+}
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
new file mode 100644
index 0000000..9deb15b
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,166 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+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.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+
+/**
+ * View to an on-disk Backup Image FileSytem
+ * Provides the set of methods necessary to interact with the on-disk Backup Image data.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HBackupFileSystem {
+ public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
+
+ /**
+ * This is utility class.
+ */
+ private HBackupFileSystem() {
+ }
+
+ /**
+ * Given the backup root dir, backup id and the table name, return the backup image location,
+ * which is also where the backup manifest file is. return value look like:
+ * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/"
+ * @param backupRootDir backup root directory
+ * @param backupId backup id
+ * @param tableName table name
+ * @return backupPath String for the particular table
+ */
+ public static String getTableBackupDir(String backupRootDir, String backupId,
+ TableName tableName) {
+ return backupRootDir + Path.SEPARATOR+ backupId + Path.SEPARATOR +
+ tableName.getNamespaceAsString() + Path.SEPARATOR
+ + tableName.getQualifierAsString() + Path.SEPARATOR ;
+ }
+
+ /**
+ * Given the backup root dir, backup id and the table name, return the backup image location,
+ * which is also where the backup manifest file is. return value look like:
+ * "hdfs://backup.hbase.org:9000/user/biadmin/backup_1396650096738/backup1/default/t1_dn/"
+ * @param backupRootPath backup root path
+ * @param tableName table name
+ * @param backupId backup Id
+ * @return backupPath for the particular table
+ */
+ public static Path getTableBackupPath(TableName tableName, Path backupRootPath, String backupId) {
+ return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName));
+ }
+
+
+ public static List loadRegionInfos(TableName tableName,
+ Path backupRootPath, String backupId, Configuration conf) throws IOException
+ {
+ Path backupTableRoot = getTableBackupPath(tableName, backupRootPath, backupId);
+ FileSystem fs = backupTableRoot.getFileSystem(conf);
+ RemoteIterator it = fs.listFiles(backupTableRoot, true);
+ List infos = new ArrayList();
+ while(it.hasNext()) {
+ LocatedFileStatus lfs = it.next();
+ if(lfs.isFile() && lfs.getPath().toString().endsWith(HRegionFileSystem.REGION_INFO_FILE)) {
+ Path regionDir = lfs.getPath().getParent();
+ HRegionInfo info = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ 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.
+ * @param backupRootDir backup root directory
+ * @param backupId backup id
+ * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
+ */
+ public static String getLogBackupDir(String backupRootDir, String backupId) {
+ return backupRootDir + Path.SEPARATOR + backupId+ Path.SEPARATOR
+ + HConstants.HREGION_LOGDIR_NAME;
+ }
+
+ public static Path getLogBackupPath(String backupRootDir, String backupId) {
+ return new Path(getLogBackupDir(backupRootDir, backupId));
+ }
+
+ private static Path getManifestPath(TableName tableName, Configuration conf,
+ 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
+ manifestPath =
+ new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR
+ + BackupManifest.MANIFEST_FILE_NAME);
+ 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 ?";
+ throw new IOException(errorMsg);
+ }
+ }
+ return manifestPath;
+ }
+
+ public static BackupManifest getManifest(TableName tableName, Configuration conf,
+ Path backupRootPath, String backupId) throws IOException {
+ BackupManifest manifest = new BackupManifest(conf,
+ getManifestPath(tableName, conf, backupRootPath, backupId));
+ return manifest;
+ }
+
+ /**
+ * Check whether the backup image path and there is manifest file in the path.
+ * @param backupManifestMap If all the manifests are found, then they are put into this map
+ * @param tableArray the tables involved
+ * @throws IOException exception
+ */
+ public static void checkImageManifestExist(HashMap backupManifestMap,
+ TableName[] tableArray, Configuration conf,
+ Path backupRootPath, String backupId) throws IOException {
+ for (TableName tableName : tableArray) {
+ BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId);
+ backupManifestMap.put(tableName, manifest);
+ }
+ }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..6fafe00
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+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.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
+import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
+import org.apache.hadoop.hbase.backup.util.LogUtils;
+import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public class RestoreDriver extends AbstractHBaseTool implements BackupRestoreConstants {
+
+ private static final Log LOG = LogFactory.getLog(RestoreDriver.class);
+ private CommandLine cmd;
+
+ private static final String USAGE_STRING =
+ "Usage: bin/hbase restore [options]\n"
+ + " backup_path Path to a backup destination root\n"
+ + " backup_id Backup image ID to restore"
+ + " table(s) Comma-separated list of tables to restore";
+
+ private static final String USAGE_FOOTER = "";
+
+ protected RestoreDriver() throws IOException {
+ init();
+ }
+
+ protected void init() throws IOException {
+ // disable irrelevant loggers to avoid it mess up command output
+ LogUtils.disableZkAndClientLoggers(LOG);
+ }
+
+ private int parseAndRun(String[] args) throws IOException {
+ // enable debug logging
+ Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+ if (cmd.hasOption(OPTION_DEBUG)) {
+ backupClientLogger.setLevel(Level.DEBUG);
+ }
+
+ // whether to overwrite to existing table if any, false by default
+ boolean overwrite = cmd.hasOption(OPTION_OVERWRITE);
+ if (overwrite) {
+ LOG.debug("Found -overwrite option in restore command, "
+ + "will overwrite to existing table if any in the restore target");
+ }
+
+ // whether to only check the dependencies, false by default
+ boolean check = cmd.hasOption(OPTION_CHECK);
+ if (check) {
+ LOG.debug("Found -check option in restore command, "
+ + "will check and verify the dependencies");
+ }
+
+ LOG.debug("Will automatically restore all the dependencies");
+
+ // parse main restore command options
+ String[] remainArgs = cmd.getArgs();
+ if (remainArgs.length < 3 && !cmd.hasOption(OPTION_SET)
+ || (cmd.hasOption(OPTION_SET) && remainArgs.length < 2)) {
+ printToolUsage();
+ return -1;
+ }
+
+ String backupRootDir = remainArgs[0];
+ String backupId = remainArgs[1];
+ String tables = null;
+ 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);) {
+ // Check backup set
+ if (cmd.hasOption(OPTION_SET)) {
+ String setName = cmd.getOptionValue(OPTION_SET);
+ try {
+ tables = getTablesForSet(conn, setName, conf);
+ } catch (IOException e) {
+ System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName);
+ printToolUsage();
+ return -2;
+ }
+ if (tables == null) {
+ System.out.println("ERROR: Backup set '" + setName
+ + "' is either empty or does not exist");
+ printToolUsage();
+ return -3;
+ }
+ } else {
+ tables = remainArgs[2];
+ }
+
+ TableName[] sTableArray = BackupServerUtil.parseTableNames(tables);
+ TableName[] tTableArray = BackupServerUtil.parseTableNames(tableMapping);
+
+ if (sTableArray != null && tTableArray != null
+ && (sTableArray.length != tTableArray.length)) {
+ System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+ printToolUsage();
+ return -4;
+ }
+
+ client.restore(RestoreServerUtil.createRestoreRequest(backupRootDir, backupId, check,
+ sTableArray, tTableArray, overwrite));
+ } catch (Exception e) {
+ e.printStackTrace();
+ return -5;
+ }
+ return 0;
+ }
+
+ private String getTablesForSet(Connection conn, String name, Configuration conf)
+ throws IOException {
+ try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+ List tables = table.describeBackupSet(name);
+ if (tables == null) return null;
+ return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+ }
+ }
+
+ @Override
+ protected void addOptions() {
+ // define supported options
+ addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC);
+ addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC);
+ addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+ addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC);
+ addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC);
+ }
+
+ @Override
+ protected void processOptions(CommandLine cmd) {
+ this.cmd = cmd;
+ }
+
+ @Override
+ protected int doWork() throws Exception {
+ return parseAndRun(cmd.getArgs());
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration conf = HBaseConfiguration.create();
+ Path hbasedir = FSUtils.getRootDir(conf);
+ URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+ FSUtils.setFsDefault(conf, new Path(defaultFs));
+ int ret = ToolRunner.run(conf, new RestoreDriver(), args);
+ System.exit(ret);
+ }
+
+ @Override
+ public int run(String[] args) throws IOException {
+ if (conf == null) {
+ LOG.error("Tool configuration is not initialized");
+ throw new NullPointerException("conf");
+ }
+
+ CommandLine cmd;
+ try {
+ // parse the command line arguments
+ cmd = parseArgs(args);
+ cmdLineArgs = args;
+ } catch (Exception e) {
+ System.out.println("Error when parsing command-line arguments: " + e.getMessage());
+ printToolUsage();
+ return EXIT_FAILURE;
+ }
+
+ if (!sanityCheckOptions(cmd) || cmd.hasOption(SHORT_HELP_OPTION)
+ || cmd.hasOption(LONG_HELP_OPTION)) {
+ printToolUsage();
+ return EXIT_FAILURE;
+ }
+
+ processOptions(cmd);
+
+ int ret = EXIT_FAILURE;
+ try {
+ ret = doWork();
+ } catch (Exception e) {
+ LOG.error("Error running command-line tool", e);
+ return EXIT_FAILURE;
+ }
+ return ret;
+ }
+
+ @Override
+ protected boolean sanityCheckOptions(CommandLine cmd) {
+ boolean success = true;
+ for (String reqOpt : requiredOptions) {
+ if (!cmd.hasOption(reqOpt)) {
+ System.out.println("Required option -" + reqOpt + " is missing");
+ success = false;
+ }
+ }
+ return success;
+ }
+
+ protected void printToolUsage() throws IOException {
+ System.out.println(USAGE_STRING);
+ HelpFormatter helpFormatter = new HelpFormatter();
+ helpFormatter.setLeftPadding(2);
+ helpFormatter.setDescPadding(8);
+ helpFormatter.setWidth(100);
+ helpFormatter.setSyntaxPrefix("Options:");
+ helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
new file mode 100644
index 0000000..7490d20
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO class for restore request
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RestoreRequest {
+
+ private String backupRootDir;
+ private String backupId;
+ private boolean check = false;
+ private TableName[] fromTables;
+ private TableName[] toTables;
+ private boolean overwrite = false;
+
+ public RestoreRequest() {
+ }
+
+ public String getBackupRootDir() {
+ return backupRootDir;
+ }
+
+ public RestoreRequest setBackupRootDir(String backupRootDir) {
+ this.backupRootDir = backupRootDir;
+ return this;
+ }
+
+ public String getBackupId() {
+ return backupId;
+ }
+
+ public RestoreRequest setBackupId(String backupId) {
+ this.backupId = backupId;
+ return this;
+ }
+
+ public boolean isCheck() {
+ return check;
+ }
+
+ public RestoreRequest setCheck(boolean check) {
+ this.check = check;
+ return this;
+ }
+
+ public TableName[] getFromTables() {
+ return fromTables;
+ }
+
+ public RestoreRequest setFromTables(TableName[] fromTables) {
+ this.fromTables = fromTables;
+ return this;
+ }
+
+ public TableName[] getToTables() {
+ return toTables;
+ }
+
+ public RestoreRequest setToTables(TableName[] toTables) {
+ this.toTables = toTables;
+ return this;
+ }
+
+ public boolean isOverwrite() {
+ return overwrite;
+ }
+
+ public RestoreRequest setOverwrite(boolean overwrite) {
+ this.overwrite = overwrite;
+ return this;
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreTask.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreTask.java
new file mode 100644
index 0000000..bd43990
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreTask.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+/**
+ * Backup restore service interface
+ * Concrete implementation is provided by backup provider.
+ */
+
+public interface RestoreTask extends Configurable{
+
+ /**
+ * Run restore operation
+ * @param dirPaths - path array of WAL log directories
+ * @param fromTables - from tables
+ * @param toTables - to tables
+ * @param fullBackupRestore - full backup restore
+ * @throws IOException
+ */
+ void run(Path[] dirPaths, TableName[] fromTables,
+ TableName[] toTables, boolean fullBackupRestore)
+ throws IOException;
+}
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
new file mode 100644
index 0000000..c4227f5
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,754 @@
+/**
+ * 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.net.URI;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+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.util.BackupClientUtil;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * General backup commands, options and usage messages
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupCommands implements BackupRestoreConstants {
+
+ public final static String INCORRECT_USAGE = "Incorrect usage";
+
+ public static final String USAGE =
+ "Usage: bin/hbase backup COMMAND [command-specific arguments]\n"
+ + "where COMMAND is one of:\n"
+ + " create create a new backup image\n"
+ + " delete delete an existing backup image\n"
+ + " describe show the detailed information of a backup image\n"
+ + " history show history of all successful backups\n"
+ + " progress show the progress of the latest backup request\n"
+ + " set backup set management\n"
+ + "Run \'bin/hbase backup COMMAND -h\' to see help message for each command\n";
+
+ public static final String CREATE_CMD_USAGE =
+ "Usage: bin/hbase backup create [tables] [options]\n"
+ + " type \"full\" to create a full backup image\n"
+ + " \"incremental\" to create an incremental backup image\n"
+ + " backup_root Full path to store the backup image\n"
+ + " tables If no tables (\"\") are specified, all tables are backed up.\n"
+ + " otherwise it is a comma separated list of tables.";
+
+
+ public static final String PROGRESS_CMD_USAGE = "Usage: bin/hbase backup progress \n"
+ + " backupId Backup image id\n";
+ public static final String NO_INFO_FOUND = "No info was found for backup id: ";
+
+ public static final String DESCRIBE_CMD_USAGE = "Usage: bin/hbase backup describe \n"
+ + " backupId Backup image id\n";
+
+ public static final String HISTORY_CMD_USAGE =
+ "Usage: bin/hbase backup history [options]";
+
+
+
+ public static final String DELETE_CMD_USAGE = "Usage: bin/hbase backup delete \n"
+ + " backupId Backup image id\n";
+
+ public static final String CANCEL_CMD_USAGE = "Usage: bin/hbase backup cancel \n"
+ + " backupId Backup image id\n";
+
+ public static final String SET_CMD_USAGE = "Usage: bin/hbase backup set COMMAND [name] [tables]\n"
+ + " name Backup set name\n"
+ + " tables If no tables (\"\") are specified, all tables will belong to the set.\n"
+ + " Otherwise it is a comma separated list of tables.\n"
+ + "COMMAND is one of:\n"
+ + " add add tables to a set, create a set if needed\n"
+ + " remove remove tables from a set\n"
+ + " list list all backup sets in the system\n"
+ + " describe describe set\n"
+ + " delete delete backup set\n";
+
+ public static final String USAGE_FOOTER = "";
+
+ public static abstract class Command extends Configured {
+ CommandLine cmdline;
+
+ Command(Configuration conf) {
+ super(conf);
+ }
+
+ public void execute() throws IOException
+ {
+ if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ }
+
+ protected abstract void printUsage();
+ }
+
+ private BackupCommands() {
+ throw new AssertionError("Instantiating utility class...");
+ }
+
+ public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) {
+ Command cmd = null;
+ switch (type) {
+ case CREATE:
+ cmd = new CreateCommand(conf, cmdline);
+ break;
+ case DESCRIBE:
+ cmd = new DescribeCommand(conf, cmdline);
+ break;
+ case PROGRESS:
+ cmd = new ProgressCommand(conf, cmdline);
+ break;
+ case DELETE:
+ cmd = new DeleteCommand(conf, cmdline);
+ break;
+ case CANCEL:
+ cmd = new CancelCommand(conf, cmdline);
+ break;
+ case HISTORY:
+ cmd = new HistoryCommand(conf, cmdline);
+ break;
+ case SET:
+ cmd = new BackupSetCommand(conf, cmdline);
+ break;
+ case HELP:
+ default:
+ cmd = new HelpCommand(conf, cmdline);
+ break;
+ }
+ return cmd;
+ }
+
+ static int numOfArgs(String[] args) {
+ if (args == null) return 0;
+ return args.length;
+ }
+
+ public static class CreateCommand extends Command {
+
+ CreateCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ if (cmdline == null || cmdline.getArgs() == null) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ String[] args = cmdline.getArgs();
+ if (args.length < 3 || args.length > 4) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
+ && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
+ System.out.println("ERROR: invalid backup type: "+ args[1]);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ if(!verifyPath(args[2])) {
+ System.out.println("ERROR: invalid backup destination: "+ args[2]);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String tables = null;
+ Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+
+ // Check if we have both: backup set and list of tables
+ if(args.length == 4 && cmdline.hasOption(OPTION_SET)) {
+ System.out.println("ERROR: You can specify either backup set or list"+
+ " of tables, but not both");
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ // Check backup set
+ String setName = null;
+ if (cmdline.hasOption(OPTION_SET)) {
+ setName = cmdline.getOptionValue(OPTION_SET);
+ tables = getTablesForSet(setName, conf);
+
+ if (tables == null) {
+ System.out.println("ERROR: Backup set '" + setName+ "' is either empty or does not exist");
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ } else {
+ tables = (args.length == 4) ? args[3] : null;
+ }
+ int bandwidth = cmdline.hasOption(OPTION_BANDWIDTH) ?
+ Integer.parseInt(cmdline.getOptionValue(OPTION_BANDWIDTH)) : -1;
+ int workers = cmdline.hasOption(OPTION_WORKERS) ?
+ Integer.parseInt(cmdline.getOptionValue(OPTION_WORKERS)) : -1;
+
+ try (Connection conn = ConnectionFactory.createConnection(getConf());
+ HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) {
+ BackupRequest request = new BackupRequest();
+ request.setBackupType(BackupType.valueOf(args[1].toUpperCase()))
+ .setTableList(tables != null?Lists.newArrayList(BackupClientUtil.parseTableNames(tables)): null)
+ .setTargetRootDir(args[2]).setWorkers(workers).setBandwidth(bandwidth)
+ .setBackupSetName(setName);
+
+ String backupId = admin.backupTables(request);
+ System.out.println("Backup session "+ backupId+" finished. Status: SUCCESS");
+ } catch (IOException e) {
+ System.out.println("Backup session finished. Status: FAILURE");
+ throw e;
+ }
+ }
+
+ private boolean verifyPath(String path) {
+ try{
+ Path p = new Path(path);
+ Configuration conf = getConf() != null? getConf():
+ HBaseConfiguration.create();
+ URI uri = p.toUri();
+ if(uri.getScheme() == null) return false;
+ FileSystem.get(uri, conf);
+ return true;
+ } catch(Exception e){
+ return false;
+ }
+ }
+
+ private String getTablesForSet(String name, Configuration conf)
+ throws IOException {
+ try (final Connection conn = ConnectionFactory.createConnection(conf);
+ final BackupSystemTable table = new BackupSystemTable(conn)) {
+ List tables = table.describeBackupSet(name);
+ if (tables == null) return null;
+ return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(CREATE_CMD_USAGE);
+ Options options = new Options();
+ options.addOption(OPTION_WORKERS, true, OPTION_WORKERS_DESC);
+ options.addOption(OPTION_BANDWIDTH, true, OPTION_BANDWIDTH_DESC);
+ options.addOption(OPTION_SET, true, OPTION_SET_RESTORE_DESC);
+
+ HelpFormatter helpFormatter = new HelpFormatter();
+ helpFormatter.setLeftPadding(2);
+ helpFormatter.setDescPadding(8);
+ helpFormatter.setWidth(100);
+ helpFormatter.setSyntaxPrefix("Options:");
+ helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+
+ }
+ }
+
+ private static class HelpCommand extends Command {
+
+ HelpCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ if (cmdline == null) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String[] args = cmdline.getArgs();
+ if (args == null || args.length == 0) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ if (args.length != 2) {
+ System.out.println("ERROR: Only supports help message of a single command type");
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String type = args[1];
+
+ if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
+ System.out.println(CREATE_CMD_USAGE);
+ } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) {
+ System.out.println(DESCRIBE_CMD_USAGE);
+ } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) {
+ System.out.println(HISTORY_CMD_USAGE);
+ } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) {
+ System.out.println(PROGRESS_CMD_USAGE);
+ } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) {
+ System.out.println(DELETE_CMD_USAGE);
+ } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
+ System.out.println(CANCEL_CMD_USAGE);
+ } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
+ System.out.println(SET_CMD_USAGE);
+ } else {
+ System.out.println("Unknown command : " + type);
+ printUsage();
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(USAGE);
+ }
+ }
+
+ private static class DescribeCommand extends Command {
+
+ DescribeCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ if (cmdline == null || cmdline.getArgs() == null) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ String[] args = cmdline.getArgs();
+ if (args.length != 2) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String backupId = args[1];
+ Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+ try (final Connection conn = ConnectionFactory.createConnection(conf);
+ final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+ BackupInfo info = sysTable.readBackupInfo(backupId);
+ if (info == null) {
+ System.out.println("ERROR: " + backupId + " does not exist");
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ System.out.println(info.getShortDescription());
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(DESCRIBE_CMD_USAGE);
+ }
+ }
+
+ private static class ProgressCommand extends Command {
+
+ ProgressCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+
+ if (cmdline == null || cmdline.getArgs() == null ||
+ cmdline.getArgs().length == 1) {
+ System.err.println("No backup id was specified, "
+ + "will retrieve the most recent (ongoing) sessions");
+ }
+ String[] args = cmdline == null ? null : cmdline.getArgs();
+ if (args != null && args.length > 2) {
+ System.err.println("ERROR: wrong number of arguments: " + args.length);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String backupId = (args == null || args.length <= 1) ? null : args[1];
+ Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+ try(final Connection conn = ConnectionFactory.createConnection(conf);
+ final BackupSystemTable sysTable = new BackupSystemTable(conn);){
+ BackupInfo info = sysTable.readBackupInfo(backupId);
+ int progress = info == null? -1: info.getProgress();
+ if(progress < 0){
+ System.out.println(NO_INFO_FOUND + backupId);
+ } else{
+ System.out.println(backupId+" progress=" + progress+"%");
+ }
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(PROGRESS_CMD_USAGE);
+ }
+ }
+
+ private static class DeleteCommand extends Command {
+
+ DeleteCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String[] args = cmdline.getArgs();
+
+ String[] backupIds = new String[args.length - 1];
+ 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);) {
+ int deleted = admin.deleteBackups(args);
+ System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+ }
+
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(DELETE_CMD_USAGE);
+ }
+ }
+
+// TODO Cancel command
+
+ private static class CancelCommand extends Command {
+
+ CancelCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+ System.out.println("No backup id(s) was specified, will use the most recent one");
+ }
+ Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+ try (final Connection conn = ConnectionFactory.createConnection(conf);
+ HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) {
+ // TODO cancel backup
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ }
+ }
+
+ private static class HistoryCommand extends Command {
+
+ private final static int DEFAULT_HISTORY_LENGTH = 10;
+
+ HistoryCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+
+ super.execute();
+
+ int n = parseHistoryLength();
+ final TableName tableName = getTableName();
+ final String setName = getTableSetName();
+ BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() {
+ @Override
+ public boolean apply(BackupInfo info) {
+ if (tableName == null) return true;
+ List names = info.getTableNames();
+ return names.contains(tableName);
+ }
+ };
+ BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() {
+ @Override
+ public boolean apply(BackupInfo info) {
+ if (setName == null) return true;
+ String backupId = info.getBackupId();
+ return backupId.startsWith(setName);
+ }
+ };
+ Path backupRootPath = getBackupRootPath();
+ List history = null;
+ Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+ if (backupRootPath == null) {
+ // Load from hbase:backup
+ try (final Connection conn = ConnectionFactory.createConnection(conf);
+ final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+
+ history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
+ }
+ } else {
+ // load from backup FS
+ history = BackupClientUtil.getHistory(conf, n, backupRootPath,
+ tableNameFilter, tableSetFilter);
+ }
+ for (BackupInfo info : history) {
+ System.out.println(info.getShortDescription());
+ }
+ }
+
+ private Path getBackupRootPath() throws IOException {
+ String value = null;
+ try{
+ value = cmdline.getOptionValue(OPTION_PATH);
+ if (value == null) return null;
+ return new Path(value);
+ } catch (IllegalArgumentException e) {
+ System.out.println("ERROR: Illegal argument for backup root path: "+ value);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ }
+
+ private TableName getTableName() throws IOException {
+ String value = cmdline.getOptionValue(OPTION_TABLE);
+ if (value == null) return null;
+ try{
+ return TableName.valueOf(value);
+ } catch (IllegalArgumentException e){
+ System.out.println("Illegal argument for table name: "+ value);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ }
+
+ private String getTableSetName() throws IOException {
+ String value = cmdline.getOptionValue(OPTION_SET);
+ return value;
+ }
+
+ private int parseHistoryLength() throws IOException {
+ String value = cmdline.getOptionValue(OPTION_RECORD_NUMBER);
+ try{
+ if (value == null) return DEFAULT_HISTORY_LENGTH;
+ return Integer.parseInt(value);
+ } catch(NumberFormatException e) {
+ System.out.println("Illegal argument for history length: "+ value);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(HISTORY_CMD_USAGE);
+ Options options = new Options();
+ options.addOption(OPTION_RECORD_NUMBER, true, OPTION_RECORD_NUMBER_DESC);
+ options.addOption(OPTION_PATH, true, OPTION_PATH_DESC);
+ options.addOption(OPTION_TABLE, true, OPTION_TABLE_DESC);
+ options.addOption(OPTION_SET, true, OPTION_SET_DESC);
+
+ HelpFormatter helpFormatter = new HelpFormatter();
+ helpFormatter.setLeftPadding(2);
+ helpFormatter.setDescPadding(8);
+ helpFormatter.setWidth(100);
+ helpFormatter.setSyntaxPrefix("Options:");
+ helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+ }
+ }
+
+ private static class BackupSetCommand extends Command {
+ private final static String SET_ADD_CMD = "add";
+ private final static String SET_REMOVE_CMD = "remove";
+ private final static String SET_DELETE_CMD = "delete";
+ private final static String SET_DESCRIBE_CMD = "describe";
+ private final static String SET_LIST_CMD = "list";
+
+ BackupSetCommand(Configuration conf, CommandLine cmdline) {
+ super(conf);
+ this.cmdline = cmdline;
+ }
+
+ @Override
+ public void execute() throws IOException {
+ super.execute();
+ // Command-line must have at least one element
+ if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String[] args = cmdline.getArgs();
+ String cmdStr = args[1];
+ BackupCommand cmd = getCommand(cmdStr);
+
+ switch (cmd) {
+ case SET_ADD:
+ processSetAdd(args);
+ break;
+ case SET_REMOVE:
+ processSetRemove(args);
+ break;
+ case SET_DELETE:
+ processSetDelete(args);
+ break;
+ case SET_DESCRIBE:
+ processSetDescribe(args);
+ break;
+ case SET_LIST:
+ processSetList(args);
+ break;
+ default:
+ break;
+
+ }
+ }
+
+ private void processSetList(String[] args) throws IOException {
+ // List all backup set names
+ // does not expect any args
+ Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+ try(final Connection conn = ConnectionFactory.createConnection(conf);
+ HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+ List list = admin.listBackupSets();
+ for(BackupSet bs: list){
+ System.out.println(bs);
+ }
+ }
+ }
+
+ private void processSetDescribe(String[] args) throws IOException {
+ if (args == null || args.length != 3) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ String setName = args[2];
+ Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+ try(final Connection conn = ConnectionFactory.createConnection(conf);
+ final BackupSystemTable sysTable = new BackupSystemTable(conn);){
+ List tables = sysTable.describeBackupSet(setName);
+ BackupSet set = tables == null? null : new BackupSet(setName, tables);
+ if(set == null) {
+ System.out.println("Set '"+setName+"' does not exist.");
+ } else{
+ System.out.println(set);
+ }
+ }
+ }
+
+ private void processSetDelete(String[] args) throws IOException {
+ if (args == null || args.length != 3) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ String setName = args[2];
+ Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+ try(final Connection conn = ConnectionFactory.createConnection(conf);
+ final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+ boolean result = admin.deleteBackupSet(setName);
+ if(result){
+ System.out.println("Delete set "+setName+" OK.");
+ } else{
+ System.out.println("Set "+setName+" does not exist");
+ }
+ }
+ }
+
+ private void processSetRemove(String[] args) throws IOException {
+ if (args == null || args.length != 4) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+
+ String setName = args[2];
+ 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);){
+ admin.removeFromBackupSet(setName, tables);
+ }
+ }
+
+ private void processSetAdd(String[] args) throws IOException {
+ if (args == null || args.length != 4) {
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ String setName = args[2];
+ String[] tables = args[3].split(",");
+ TableName[] tableNames = new TableName[tables.length];
+ for(int i=0; i < tables.length; i++){
+ tableNames[i] = TableName.valueOf(tables[i]);
+ }
+ Configuration conf = getConf() != null? getConf():HBaseConfiguration.create();
+ try(final Connection conn = ConnectionFactory.createConnection(conf);
+ final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+ admin.addToBackupSet(setName, tableNames);
+ }
+
+ }
+
+ private BackupCommand getCommand(String cmdStr) throws IOException {
+ if (cmdStr.equals(SET_ADD_CMD)) {
+ return BackupCommand.SET_ADD;
+ } else if (cmdStr.equals(SET_REMOVE_CMD)) {
+ return BackupCommand.SET_REMOVE;
+ } else if (cmdStr.equals(SET_DELETE_CMD)) {
+ return BackupCommand.SET_DELETE;
+ } else if (cmdStr.equals(SET_DESCRIBE_CMD)) {
+ return BackupCommand.SET_DESCRIBE;
+ } else if (cmdStr.equals(SET_LIST_CMD)) {
+ return BackupCommand.SET_LIST;
+ } else {
+ System.out.println("ERROR: Unknown command for 'set' :" + cmdStr);
+ printUsage();
+ throw new IOException(INCORRECT_USAGE);
+ }
+ }
+
+ @Override
+ protected void printUsage() {
+ System.out.println(SET_CMD_USAGE);
+ }
+
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..ca204b4
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
@@ -0,0 +1,86 @@
+/**
+ * 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 org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupException extends HBaseIOException {
+ private BackupInfo description;
+
+ /**
+ * Some exception happened for a backup and don't even know the backup that it was about
+ * @param msg Full description of the failure
+ */
+ public BackupException(String msg) {
+ super(msg);
+ }
+
+ /**
+ * Some exception happened for a backup with a cause
+ * @param cause the cause
+ */
+ public BackupException(Throwable cause) {
+ super(cause);
+ }
+
+ /**
+ * Exception for the given backup that has no previous root cause
+ * @param msg reason why the backup failed
+ * @param desc description of the backup that is being failed
+ */
+ public BackupException(String msg, BackupInfo desc) {
+ super(msg);
+ this.description = desc;
+ }
+
+ /**
+ * Exception for the given backup due to another exception
+ * @param msg reason why the backup failed
+ * @param cause root cause of the failure
+ * @param desc description of the backup that is being failed
+ */
+ public BackupException(String msg, Throwable cause, BackupInfo desc) {
+ super(msg, cause);
+ this.description = desc;
+ }
+
+ /**
+ * Exception when the description of the backup cannot be determined, due to some other root
+ * cause
+ * @param message description of what caused the failure
+ * @param e root cause
+ */
+ public BackupException(String message, Exception e) {
+ super(message, e);
+ }
+
+ public BackupInfo getBackupContext() {
+ return this.description;
+ }
+
+}
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
new file mode 100644
index 0000000..0d6320c
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,502 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+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;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+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.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.backup.master.BackupController;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+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 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
+ * incremental backup. Creates BackupContext and DispatchRequest.
+ */
+@InterfaceAudience.Private
+@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;
+
+ /**
+ * Backup manager constructor.
+ * @param conf configuration
+ * @throws IOException exception
+ */
+ public BackupManager(Connection conn, Configuration conf) throws IOException {
+ if (!conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+ BackupRestoreConstants.BACKUP_ENABLE_DEFAULT)) {
+ throw new BackupException("HBase backup is not enabled. Check your " +
+ BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting.");
+ }
+ this.conf = conf;
+ this.conn = conn;
+ this.systemTable = new BackupSystemTable(conn);
+
+ }
+
+ /**
+ * Return backup context
+ */
+ protected BackupInfo getBackupContext()
+ {
+ return backupContext;
+ }
+ /**
+ * This method modifies the master's configuration in order to inject backup-related features
+ * @param conf configuration
+ */
+ public static void decorateMasterConfiguration(Configuration conf) {
+ if (!isBackupEnabled(conf)) {
+ return;
+ }
+ // Add WAL archive cleaner plug-in
+ String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
+ String cleanerClass = BackupLogCleaner.class.getCanonicalName();
+ if (!plugins.contains(cleanerClass)) {
+ conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
+ }
+
+ String classes = conf.get("hbase.procedure.master.classes");
+ String masterProcedureClass = LogRollMasterProcedureManager.class.getName();
+ if(classes == null){
+ conf.set("hbase.procedure.master.classes", masterProcedureClass);
+ } else if(!classes.contains(masterProcedureClass)){
+ conf.set("hbase.procedure.master.classes", classes +","+masterProcedureClass);
+ }
+
+ // Set Master Observer - Backup Controller
+ classes = conf.get("hbase.coprocessor.master.classes");
+ String observerClass = BackupController.class.getName();
+ if(classes == null){
+ conf.set("hbase.coprocessor.master.classes", observerClass);
+ } else if(!classes.contains(observerClass)){
+ conf.set("hbase.coprocessor.master.classes", classes +","+observerClass);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Added log cleaner: " + cleanerClass);
+ LOG.debug("Added master procedure manager: "+masterProcedureClass);
+ LOG.debug("Added master observer: "+observerClass);
+ }
+
+ }
+
+ /**
+ * This method modifies the RS configuration in order to inject backup-related features
+ * @param conf configuration
+ */
+ public static void decorateRSConfiguration(Configuration conf) {
+ if (!isBackupEnabled(conf)) {
+ return;
+ }
+
+ String classes = conf.get("hbase.procedure.regionserver.classes");
+ String regionProcedureClass = LogRollRegionServerProcedureManager.class.getName();
+ if(classes == null){
+ conf.set("hbase.procedure.regionserver.classes", regionProcedureClass);
+ } else if(!classes.contains(regionProcedureClass)){
+ conf.set("hbase.procedure.regionserver.classes", classes +","+regionProcedureClass);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Added region procedure manager: "+regionProcedureClass);
+ }
+
+ }
+
+
+ private static boolean isBackupEnabled(Configuration conf) {
+ return conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+ BackupRestoreConstants.BACKUP_ENABLE_DEFAULT);
+ }
+
+ /**
+ * Get configuration
+ * @return configuration
+ */
+ Configuration getConf() {
+ return conf;
+ }
+
+ /**
+ * Stop all the work of backup.
+ */
+ @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();
+ } catch (Exception e) {
+ LOG.error(e);
+ }
+ }
+ if (conn != null) {
+ try {
+ conn.close();
+ } catch (IOException e) {
+ LOG.error(e);
+ }
+ }
+ }
+
+ /**
+ * Create a BackupContext based on input backup request.
+ * @param backupId backup id
+ * @param type type
+ * @param tableList table list
+ * @param targetRootDir root dir
+ * @param workers number of parallel workers
+ * @param bandwidth bandwidth per worker in MB per sec
+ * @return BackupInfo
+ * @throws BackupException exception
+ */
+ public BackupInfo createBackupContext(String backupId, BackupType type,
+ List tableList, String targetRootDir, int workers, long bandwidth)
+ throws BackupException {
+ if (targetRootDir == null) {
+ throw new BackupException("Wrong backup request parameter: target backup root directory");
+ }
+
+ if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
+ // If table list is null for full backup, which means backup all tables. Then fill the table
+ // list with all user tables from meta. It no table available, throw the request exception.
+
+ HTableDescriptor[] htds = null;
+ try (Admin admin = conn.getAdmin()) {
+ htds = admin.listTables();
+ } catch (Exception e) {
+ throw new BackupException(e);
+ }
+
+ if (htds == null) {
+ throw new BackupException("No table exists for full backup of all tables.");
+ } else {
+ tableList = new ArrayList<>();
+ for (HTableDescriptor hTableDescriptor : htds) {
+ tableList.add(hTableDescriptor.getTableName());
+ }
+
+ LOG.info("Full backup all the tables available in the cluster: " + tableList);
+ }
+ }
+
+ // there are one or more tables in the table list
+ backupContext = new BackupInfo(backupId, type,
+ tableList.toArray(new TableName[tableList.size()]),
+ targetRootDir);
+ backupContext.setBandwidth(bandwidth);
+ backupContext.setWorkers(workers);
+ return backupContext;
+ }
+
+ /**
+ * Check if any ongoing backup. Currently, we only reply on checking status in hbase:backup. We
+ * need to consider to handle the case of orphan records in the future. Otherwise, all the coming
+ * request will fail.
+ * @return the ongoing backup id if on going backup exists, otherwise null
+ * @throws IOException exception
+ */
+ private String getOngoingBackupId() throws IOException {
+
+ ArrayList sessions = systemTable.getBackupContexts(BackupState.RUNNING);
+ if (sessions.size() == 0) {
+ return null;
+ }
+ return sessions.get(0).getBackupId();
+ }
+
+ /**
+ * Start the backup manager service.
+ * @throws IOException exception
+ */
+ public void initialize() throws IOException {
+ String ongoingBackupId = this.getOngoingBackupId();
+ if (ongoingBackupId != null) {
+ LOG.info("There is a ongoing backup " + ongoingBackupId
+ + ". 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) {
+ this.backupContext = backupContext;
+ }
+
+ /**
+ * Get direct ancestors of the current backup.
+ * @param backupCtx The backup context for the current backup
+ * @return The ancestors for the current backup
+ * @throws IOException exception
+ * @throws BackupException exception
+ */
+ public ArrayList getAncestors(BackupInfo backupCtx) throws IOException,
+ BackupException {
+ LOG.debug("Getting the direct ancestors of the current backup "+
+ backupCtx.getBackupId());
+
+ ArrayList ancestors = new ArrayList();
+
+ // full backup does not have ancestor
+ if (backupCtx.getType() == BackupType.FULL) {
+ LOG.debug("Current backup is a full backup, no direct ancestor for it.");
+ return ancestors;
+ }
+
+ // get all backup history list in descending order
+
+ ArrayList allHistoryList = getBackupHistory(true);
+ for (BackupInfo backup : allHistoryList) {
+ BackupImage image =
+ new BackupImage(backup.getBackupId(), backup.getType(),
+ backup.getTargetRootDir(),
+ backup.getTableNames(), backup.getStartTs(), backup
+ .getEndTs());
+ // 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,
+ // then no need to add
+ if (!BackupManifest.canCoverImage(ancestors, image)) {
+ ancestors.add(image);
+ }
+ } else {
+ // found last incremental backup, if previously added full backup ancestor images can cover
+ // it, then this incremental ancestor is not the dependent of the current incremental
+ // backup, that is to say, this is the backup scope boundary of current table set.
+ // 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:");
+ for (BackupImage image1 : ancestors) {
+ LOG.debug(" BackupId: " + image1.getBackupId() + ", Backup directory: "
+ + image1.getRootDir());
+ }
+ } else {
+ Path logBackupPath =
+ HBackupFileSystem.getLogBackupPath(backup.getTargetRootDir(),
+ backup.getBackupId());
+ LOG.debug("Current backup has an incremental backup ancestor, "
+ + "touching its image manifest in " + logBackupPath.toString()
+ + " to construct the dependency.");
+ BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath);
+ 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("Got " + ancestors.size() + " ancestors for the current backup.");
+ return ancestors;
+ }
+
+ /**
+ * Get the direct ancestors of this backup for one table involved.
+ * @param backupContext backup context
+ * @param table table
+ * @return backupImages on the dependency list
+ * @throws BackupException exception
+ * @throws IOException exception
+ */
+ public ArrayList getAncestors(BackupInfo backupContext, TableName table)
+ throws BackupException, IOException {
+ ArrayList ancestors = getAncestors(backupContext);
+ ArrayList tableAncestors = new ArrayList();
+ for (BackupImage image : ancestors) {
+ if (image.hasTable(table)) {
+ tableAncestors.add(image);
+ if (image.getType() == BackupType.FULL) {
+ break;
+ }
+ }
+ }
+ return tableAncestors;
+ }
+
+ /*
+ * hbase:backup operations
+ */
+
+ /**
+ * Updates status (state) of a backup session in a persistent store
+ * @param context context
+ * @throws IOException exception
+ */
+ public void updateBackupInfo(BackupInfo context) throws IOException {
+ systemTable.updateBackupInfo(context);
+ }
+
+ /**
+ * Read the last backup start code (timestamp) of last successful backup. Will return null
+ * if there is no startcode stored in hbase:backup or the value is of length 0. These two
+ * cases indicate there is no successful backup completed so far.
+ * @return the timestamp of a last successful backup
+ * @throws IOException exception
+ */
+ public String readBackupStartCode() throws IOException {
+ return systemTable.readBackupStartCode(backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+ * @param startCode start code
+ * @throws IOException exception
+ */
+ public void writeBackupStartCode(Long startCode) throws IOException {
+ systemTable.writeBackupStartCode(startCode, backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Get the RS log information after the last log roll from hbase:backup.
+ * @return RS log info
+ * @throws IOException exception
+ */
+ public HashMap readRegionServerLastLogRollResult() throws IOException {
+ return systemTable.readRegionServerLastLogRollResult(backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Get all completed backup information (in desc order by time)
+ * @return history info of BackupCompleteData
+ * @throws IOException exception
+ */
+ public List getBackupHistory() throws IOException {
+ return systemTable.getBackupHistory();
+ }
+
+ public ArrayList getBackupHistory(boolean completed) throws IOException {
+ return systemTable.getBackupHistory(completed);
+ }
+ /**
+ * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+ * incremental backup. Each table may have a different set of log timestamps. The saved timestamp
+ * is of the last log file that was backed up already.
+ * @param tables tables
+ * @throws IOException exception
+ */
+ public void writeRegionServerLogTimestamp(Set tables,
+ HashMap newTimestamps) throws IOException {
+ systemTable.writeRegionServerLogTimestamp(tables, newTimestamps,
+ backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Read the timestamp for each region server log after the last successful backup. Each table has
+ * its own set of the timestamps.
+ * @return the timestamp for each region server. key: tableName value:
+ * RegionServer,PreviousTimeStamp
+ * @throws IOException exception
+ */
+ public HashMap> readLogTimestampMap() throws IOException {
+ return systemTable.readLogTimestampMap(backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Return the current tables covered by incremental backup.
+ * @return set of tableNames
+ * @throws IOException exception
+ */
+ public Set getIncrementalBackupTableSet() throws IOException {
+ return systemTable.getIncrementalBackupTableSet(backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Adds set of tables to overall incremental backup table set
+ * @param tables tables
+ * @throws IOException exception
+ */
+ public void addIncrementalBackupTableSet(Set tables) throws IOException {
+ systemTable.addIncrementalBackupTableSet(tables, backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Saves list of WAL files after incremental backup operation. These files will be stored until
+ * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be
+ * safely purged.
+ */
+ public void recordWALFiles(List files) throws IOException {
+ systemTable.addWALFiles(files,
+ backupContext.getBackupId(), backupContext.getTargetRootDir());
+ }
+
+ /**
+ * Get WAL files iterator
+ * @return WAL files iterator from hbase:backup
+ * @throws IOException
+ */
+ public Iterator getWALFilesFromBackupSystem() throws IOException {
+ return systemTable.getWALFilesIterator(backupContext.getTargetRootDir());
+ }
+
+ public Connection getConnection() {
+ return conn;
+ }
+}
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
new file mode 100644
index 0000000..c3dc539
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,711 @@
+/**
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+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;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+
+/**
+ * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
+ * as manifest file together with data. So that each backup image will contain all the info needed
+ * for restore.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManifest {
+
+ private static final Log LOG = LogFactory.getLog(BackupManifest.class);
+
+ // manifest file name
+ public static final String MANIFEST_FILE_NAME = ".backup.manifest";
+
+ // backup image, the dependency graph is made up by series of backup images
+
+ public static class BackupImage implements Comparable {
+
+ private String backupId;
+ private BackupType type;
+ private String rootDir;
+ private List tableList;
+ private long startTs;
+ private long completeTs;
+ private ArrayList ancestors;
+ private HashMap> incrTimeRanges;
+
+ public BackupImage() {
+ super();
+ }
+
+ public BackupImage(String backupId, BackupType type, String rootDir,
+ List tableList, long startTs, long completeTs) {
+ this.backupId = backupId;
+ this.type = type;
+ this.rootDir = rootDir;
+ this.tableList = tableList;
+ this.startTs = startTs;
+ this.completeTs = completeTs;
+ }
+
+ static BackupImage fromProto(BackupProtos.BackupImage im) {
+ String backupId = im.getBackupId();
+ String rootDir = im.getRootDir();
+ long startTs = im.getStartTs();
+ long completeTs = im.getCompleteTs();
+ List tableListList = im.getTableListList();
+ List tableList = new ArrayList();
+ 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;
+
+ BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
+ for(BackupProtos.BackupImage img: ancestorList) {
+ image.addAncestor(fromProto(img));
+ }
+ image.setIncrTimeRanges(loadIncrementalTimestampMap(im));
+ return image;
+ }
+
+ BackupProtos.BackupImage toProto() {
+ BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
+ builder.setBackupId(backupId);
+ builder.setCompleteTs(completeTs);
+ builder.setStartTs(startTs);
+ builder.setRootDir(rootDir);
+ if (type == BackupType.FULL) {
+ builder.setBackupType(BackupProtos.BackupType.FULL);
+ } else{
+ builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
+ }
+
+ for (TableName name: tableList) {
+ builder.addTableList(ProtobufUtil.toProtoTableNameShaded(name));
+ }
+
+ if (ancestors != null){
+ for (BackupImage im: ancestors){
+ builder.addAncestors(im.toProto());
+ }
+ }
+
+ setIncrementalTimestampMap(builder);
+ return builder.build();
+ }
+
+
+ private static HashMap>
+ loadIncrementalTimestampMap(BackupProtos.BackupImage proto) {
+ List list = proto.getTstMapList();
+
+ HashMap> incrTimeRanges =
+ new HashMap>();
+ if(list == null || list.size() == 0) return incrTimeRanges;
+ for(BackupProtos.TableServerTimestamp tst: list){
+ TableName tn = ProtobufUtil.toTableName(tst.getTable());
+ HashMap map = incrTimeRanges.get(tn);
+ if(map == null){
+ map = new HashMap();
+ incrTimeRanges.put(tn, map);
+ }
+ List listSt = tst.getServerTimestampList();
+ for(BackupProtos.ServerTimestamp stm: listSt) {
+ ServerName sn = ProtobufUtil.toServerNameShaded(stm.getServer());
+ map.put(sn.getHostname() +":" + sn.getPort(), stm.getTimestamp());
+ }
+ }
+ return incrTimeRanges;
+ }
+
+
+ private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) {
+ if (this.incrTimeRanges == null) {
+ return;
+ }
+ for (Entry> entry: this.incrTimeRanges.entrySet()) {
+ TableName key = entry.getKey();
+ HashMap value = entry.getValue();
+ BackupProtos.TableServerTimestamp.Builder tstBuilder =
+ BackupProtos.TableServerTimestamp.newBuilder();
+ tstBuilder.setTable(ProtobufUtil.toProtoTableNameShaded(key));
+
+ for (Map.Entry entry2 : value.entrySet()) {
+ String s = entry2.getKey();
+ BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
+ HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
+ ServerName sn = ServerName.parseServerName(s);
+ snBuilder.setHostName(sn.getHostname());
+ snBuilder.setPort(sn.getPort());
+ stBuilder.setServer(snBuilder.build());
+ stBuilder.setTimestamp(entry2.getValue());
+ tstBuilder.addServerTimestamp(stBuilder.build());
+ }
+ builder.addTstMap(tstBuilder.build());
+ }
+ }
+
+ public String getBackupId() {
+ return backupId;
+ }
+
+ public void setBackupId(String backupId) {
+ this.backupId = backupId;
+ }
+
+ public BackupType getType() {
+ return type;
+ }
+
+ public void setType(BackupType type) {
+ this.type = type;
+ }
+
+ public String getRootDir() {
+ return rootDir;
+ }
+
+ public void setRootDir(String rootDir) {
+ this.rootDir = rootDir;
+ }
+
+ public List getTableNames() {
+ return tableList;
+ }
+
+ public void setTableList(List tableList) {
+ this.tableList = tableList;
+ }
+
+ public long getStartTs() {
+ return startTs;
+ }
+
+ public void setStartTs(long startTs) {
+ this.startTs = startTs;
+ }
+
+ public long getCompleteTs() {
+ return completeTs;
+ }
+
+ public void setCompleteTs(long completeTs) {
+ this.completeTs = completeTs;
+ }
+
+ public ArrayList getAncestors() {
+ if (this.ancestors == null) {
+ this.ancestors = new ArrayList();
+ }
+ return this.ancestors;
+ }
+
+ public void addAncestor(BackupImage backupImage) {
+ this.getAncestors().add(backupImage);
+ }
+
+ public boolean hasAncestor(String token) {
+ for (BackupImage image : this.getAncestors()) {
+ if (image.getBackupId().equals(token)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public boolean hasTable(TableName table) {
+ for (TableName t : tableList) {
+ if (t.equals(table)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public int compareTo(BackupImage other) {
+ String thisBackupId = this.getBackupId();
+ String otherBackupId = other.getBackupId();
+ int index1 = thisBackupId.lastIndexOf("_");
+ int index2 = otherBackupId.lastIndexOf("_");
+ String name1 = thisBackupId.substring(0, index1);
+ String name2 = otherBackupId.substring(0, index2);
+ if(name1.equals(name2)) {
+ Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1));
+ Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1));
+ return thisTS.compareTo(otherTS);
+ } else {
+ return name1.compareTo(name2);
+ }
+ }
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof BackupImage) {
+ return this.compareTo((BackupImage)obj) == 0;
+ }
+ return false;
+ }
+ @Override
+ public int hashCode() {
+ int hash = 33 * this.getBackupId().hashCode() + type.hashCode();
+ hash = 33 * hash + rootDir.hashCode();
+ hash = 33 * hash + Long.valueOf(startTs).hashCode();
+ hash = 33 * hash + Long.valueOf(completeTs).hashCode();
+ for (TableName table : tableList) {
+ hash = 33 * hash + table.hashCode();
+ }
+ return hash;
+ }
+
+ public HashMap> getIncrTimeRanges() {
+ return incrTimeRanges;
+ }
+
+ public void setIncrTimeRanges(HashMap> incrTimeRanges) {
+ this.incrTimeRanges = incrTimeRanges;
+ }
+ }
+
+ // 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
+ */
+ 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);
+ }
+
+
+ /**
+ * Construct a table level manifest for a backup of the named table.
+ * @param backupCtx The ongoing backup context
+ */
+ 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();
+ 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);
+ }
+
+ /**
+ * Construct manifest from a backup directory.
+ * @param conf configuration
+ * @param backupPath backup path
+ * @throws IOException
+ */
+
+ public BackupManifest(Configuration conf, Path backupPath) throws IOException {
+ this(backupPath.getFileSystem(conf), backupPath);
+ }
+
+ /**
+ * Construct manifest from a backup directory.
+ * @param fs the FileSystem
+ * @param backupPath backup path
+ * @throws BackupException exception
+ */
+
+ public BackupManifest(FileSystem fs, Path backupPath) throws BackupException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Loading manifest from: " + backupPath.toString());
+ }
+ // The input backupDir may not exactly be the backup table dir.
+ // 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);
+ if (subFiles == null) {
+ String errorMsg = backupPath.toString() + " does not exist";
+ LOG.error(errorMsg);
+ throw new IOException(errorMsg);
+ }
+ for (FileStatus subFile : subFiles) {
+ if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
+
+ // load and set manifest field from file content
+ FSDataInputStream in = fs.open(subFile.getPath());
+ long len = subFile.getLen();
+ byte[] pbBytes = new byte[(int) len];
+ in.readFully(pbBytes);
+ BackupProtos.BackupImage proto = null;
+ try{
+ proto = BackupProtos.BackupImage.parseFrom(pbBytes);
+ } catch(Exception e){
+ 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;
+ }
+ }
+ String errorMsg = "No manifest file found in: " + backupPath.toString();
+ throw new IOException(errorMsg);
+
+ } catch (IOException e) {
+ throw new BackupException(e.getMessage());
+ }
+ }
+
+ 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.");
+ }
+
+ /**
+ * 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;
+ }
+
+ /**
+ * Persist the manifest file.
+ * @throws IOException IOException when storing the manifest file.
+ */
+
+ public void store(Configuration conf) throws BackupException {
+ byte[] data = backupImage.toProto().toByteArray();
+ // write the file, overwrite if already exist
+ Path manifestFilePath =
+ new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
+ ,MANIFEST_FILE_NAME);
+ try {
+ FSDataOutputStream out =
+ manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
+ out.write(data);
+ out.close();
+ } catch (IOException e) {
+ throw new BackupException(e.getMessage());
+ }
+
+ LOG.info("Manifest file stored to " + manifestFilePath);
+ }
+
+
+ /**
+ * Get this backup image.
+ * @return the backup image.
+ */
+ public BackupImage getBackupImage() {
+ return backupImage;
+ }
+
+ /**
+ * Add dependent backup image for this backup.
+ * @param image The direct dependent backup image
+ */
+ public void addDependentImage(BackupImage image) {
+ this.backupImage.addAncestor(image);
+ }
+
+ /**
+ * Set the incremental timestamp map directly.
+ * @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;
+ }
+
+ /**
+ * Get the image list of this backup for restore in time order.
+ * @param reverse If true, then output in reverse order, otherwise in time order from old to new
+ * @return the backup image list for restore in time order
+ */
+ public ArrayList getRestoreDependentList(boolean reverse) {
+ TreeMap restoreImages = new TreeMap();
+ restoreImages.put(backupImage.startTs, backupImage);
+ for (BackupImage image : backupImage.getAncestors()) {
+ restoreImages.put(Long.valueOf(image.startTs), image);
+ }
+ return new ArrayList(reverse ? (restoreImages.descendingMap().values())
+ : (restoreImages.values()));
+ }
+
+ /**
+ * Get the dependent image list for a specific table of this backup in time order from old to new
+ * if want to restore to this backup image level.
+ * @param table table
+ * @return the backup image list for a table in time order
+ */
+ public ArrayList getDependentListByTable(TableName table) {
+ ArrayList tableImageList = new ArrayList();
+ ArrayList imageList = getRestoreDependentList(true);
+ for (BackupImage image : imageList) {
+ if (image.hasTable(table)) {
+ tableImageList.add(image);
+ if (image.getType() == BackupType.FULL) {
+ break;
+ }
+ }
+ }
+ Collections.reverse(tableImageList);
+ return tableImageList;
+ }
+
+ /**
+ * Get the full dependent image list in the whole dependency scope for a specific table of this
+ * backup in time order from old to new.
+ * @param table table
+ * @return the full backup image list for a table in time order in the whole scope of the
+ * dependency of this image
+ */
+ public ArrayList getAllDependentListByTable(TableName table) {
+ ArrayList tableImageList = new ArrayList();
+ ArrayList imageList = getRestoreDependentList(false);
+ for (BackupImage image : imageList) {
+ if (image.hasTable(table)) {
+ tableImageList.add(image);
+ }
+ }
+ return tableImageList;
+ }
+
+ /**
+ * Check whether backup image1 could cover backup image2 or not.
+ * @param image1 backup image 1
+ * @param image2 backup image 2
+ * @return true if image1 can cover image2, otherwise false
+ */
+ public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
+ // image1 can cover image2 only when the following conditions are satisfied:
+ // - image1 must not be an incremental image;
+ // - image1 must be taken after image2 has been taken;
+ // - table set of image1 must cover the table set of image2.
+ if (image1.getType() == BackupType.INCREMENTAL) {
+ return false;
+ }
+ if (image1.getStartTs() < image2.getStartTs()) {
+ return false;
+ }
+ List image1TableList = image1.getTableNames();
+ List image2TableList = image2.getTableNames();
+ boolean found = false;
+ for (int i = 0; i < image2TableList.size(); i++) {
+ found = false;
+ for (int j = 0; j < image1TableList.size(); j++) {
+ if (image2TableList.get(i).equals(image1TableList.get(j))) {
+ found = true;
+ break;
+ }
+ }
+ if (!found) {
+ return false;
+ }
+ }
+
+ LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
+ return true;
+ }
+
+ /**
+ * Check whether backup image set could cover a backup image or not.
+ * @param fullImages The backup image set
+ * @param image The target backup image
+ * @return true if fullImages can cover image, otherwise false
+ */
+ public static boolean canCoverImage(ArrayList fullImages, BackupImage image) {
+ // fullImages can cover image only when the following conditions are satisfied:
+ // - each image of fullImages must not be an incremental image;
+ // - each image of fullImages must be taken after image has been taken;
+ // - sum table set of fullImages must cover the table set of image.
+ for (BackupImage image1 : fullImages) {
+ if (image1.getType() == BackupType.INCREMENTAL) {
+ return false;
+ }
+ if (image1.getStartTs() < image.getStartTs()) {
+ return false;
+ }
+ }
+
+ ArrayList image1TableList = new ArrayList();
+ for (BackupImage image1 : fullImages) {
+ List tableList = image1.getTableNames();
+ for (TableName table : tableList) {
+ image1TableList.add(table.getNameAsString());
+ }
+ }
+ ArrayList image2TableList = new ArrayList();
+ List tableList = image.getTableNames();
+ for (TableName table : tableList) {
+ image2TableList.add(table.getNameAsString());
+ }
+
+ for (int i = 0; i < image2TableList.size(); i++) {
+ if (image1TableList.contains(image2TableList.get(i)) == false) {
+ return false;
+ }
+ }
+
+ 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);
+ }
+ return info;
+ }
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
new file mode 100644
index 0000000..731ccd3
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
@@ -0,0 +1,36 @@
+/**
+ * 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 org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+
+/* this class will be extended in future jira to support progress report */
+public class BackupSnapshotCopy extends ExportSnapshot {
+ private String table;
+
+ public BackupSnapshotCopy(String table) {
+ super();
+ this.table = table;
+ }
+
+ public String getTable() {
+ return this.table;
+ }
+
+}
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
new file mode 100644
index 0000000..ab3f0f6
--- /dev/null
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,937 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+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.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+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.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This class provides 'hbase:backup' table API
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTable implements Closeable {
+
+ static class WALItem {
+ String backupId;
+ String walFile;
+ String backupRoot;
+
+ WALItem(String backupId, String walFile, String backupRoot) {
+ this.backupId = backupId;
+ this.walFile = walFile;
+ this.backupRoot = backupRoot;
+ }
+
+ public String getBackupId() {
+ return backupId;
+ }
+
+ public String getWalFile() {
+ return walFile;
+ }
+
+ public String getBackupRoot() {
+ return backupRoot;
+ }
+
+ @Override
+ public String toString() {
+ return "/" + backupRoot + "/" + backupId + "/" + walFile;
+ }
+
+ }
+
+ private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+ private final static TableName tableName = TableName.BACKUP_TABLE_NAME;
+ // Stores backup sessions (contexts)
+ final static byte[] SESSIONS_FAMILY = "session".getBytes();
+ // Stores other meta
+ final static byte[] META_FAMILY = "meta".getBytes();
+ // Connection to HBase cluster, shared
+ // among all instances
+ private final Connection connection;
+
+ public BackupSystemTable(Connection conn) throws IOException {
+ this.connection = conn;
+ }
+
+ @Override
+ public void close() {
+ // do nothing
+ }
+
+ /**
+ * Updates status (state) of a backup session in hbase:backup table
+ * @param context context
+ * @throws IOException exception
+ */
+ public void updateBackupInfo(BackupInfo context) throws IOException {
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
+ + " set status=" + context.getState());
+ }
+ try (Table table = connection.getTable(tableName)) {
+ Put put = BackupSystemTableHelper.createPutForBackupContext(context);
+ table.put(put);
+ }
+ }
+
+ /**
+ * Deletes backup status from hbase:backup table
+ * @param backupId backup id
+ * @throws IOException exception
+ */
+
+ public void deleteBackupInfo(String backupId) throws IOException {
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("delete backup status in hbase:backup for " + backupId);
+ }
+ try (Table table = connection.getTable(tableName)) {
+ Delete del = BackupSystemTableHelper.createDeleteForBackupInfo(backupId);
+ table.delete(del);
+ }
+ }
+
+ /**
+ * Reads backup status object (instance of BackupContext) from hbase:backup table
+ * @param backupId - backupId
+ * @return Current status of backup session or null
+ */
+
+ public BackupInfo readBackupInfo(String backupId) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("read backup status from hbase:backup for: " + backupId);
+ }
+
+ try (Table table = connection.getTable(tableName)) {
+ Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
+ Result res = table.get(get);
+ if (res.isEmpty()) {
+ return null;
+ }
+ return BackupSystemTableHelper.resultToBackupInfo(res);
+ }
+ }
+
+ /**
+ * Read the last backup start code (timestamp) of last successful backup. Will return null if
+ * there is no start code stored on hbase or the value is of length 0. These two cases indicate
+ * there is no successful backup completed so far.
+ * @param backupRoot root directory path to backup
+ * @return the timestamp of last successful backup
+ * @throws IOException exception
+ */
+ public String readBackupStartCode(String backupRoot) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("read backup start code from hbase:backup");
+ }
+ try (Table table = connection.getTable(tableName)) {
+ Get get = BackupSystemTableHelper.createGetForStartCode(backupRoot);
+ Result res = table.get(get);
+ if (res.isEmpty()) {
+ return null;
+ }
+ Cell cell = res.listCells().get(0);
+ byte[] val = CellUtil.cloneValue(cell);
+ if (val.length == 0) {
+ return null;
+ }
+ return new String(val);
+ }
+ }
+
+ /**
+ * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+ * @param startCode start code
+ * @param backupRoot root directory path to backup
+ * @throws IOException exception
+ */
+ public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("write backup start code to hbase:backup " + startCode);
+ }
+ try (Table table = connection.getTable(tableName)) {
+ Put put = BackupSystemTableHelper.createPutForStartCode(startCode.toString(), backupRoot);
+ table.put(put);
+ }
+ }
+
+ /**
+ * Get the Region Servers log information after the last log roll from hbase:backup.
+ * @param backupRoot root directory path to backup
+ * @return RS log info
+ * @throws IOException exception
+ */
+ public HashMap readRegionServerLastLogRollResult(String backupRoot)
+ throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("read region server last roll log result to hbase:backup");
+ }
+
+ Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult(backupRoot);
+
+ try (Table table = connection.getTable(tableName);
+ ResultScanner scanner = table.getScanner(scan)) {
+ Result res = null;
+ HashMap rsTimestampMap = new HashMap();
+ while ((res = scanner.next()) != null) {
+ res.advance();
+ Cell cell = res.current();
+ byte[] row = CellUtil.cloneRow(cell);
+ String server =
+ BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
+ byte[] data = CellUtil.cloneValue(cell);
+ rsTimestampMap.put(server, Bytes.toLong(data));
+ }
+ return rsTimestampMap;
+ }
+ }
+
+ /**
+ * Writes Region Server last roll log result (timestamp) to hbase:backup table
+ * @param server - Region Server name
+ * @param ts - last log timestamp
+ * @param backupRoot root directory path to backup
+ * @throws IOException exception
+ */
+ public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot)
+ throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("write region server last roll log result to hbase:backup");
+ }
+ try (Table table = connection.getTable(tableName)) {
+ Put put =
+ BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
+ table.put(put);
+ }
+ }
+
+ /**
+ * Get all completed backup information (in desc order by time)
+ * @param onlyCompleted true, if only successfully completed sessions
+ * @return history info of BackupCompleteData
+ * @throws IOException exception
+ */
+ public ArrayList getBackupHistory(boolean onlyCompleted) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("get backup history from hbase:backup");
+ }
+ ArrayList list;
+ BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY;
+ list = getBackupContexts(state);
+ return BackupClientUtil.sortHistoryListDesc(list);
+ }
+
+ /**
+ * Get all backups history
+ * @return list of backup info
+ * @throws IOException
+ */
+ public List getBackupHistory() throws IOException {
+ return getBackupHistory(false);
+ }
+
+ /**
+ * Get first n backup history records
+ * @param n - number of records
+ * @return list of records
+ * @throws IOException
+ */
+ public List getHistory(int n) throws IOException {
+
+ List history = 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;
+
+ }
+
+ /**
+ * Get backup history records filtered by list
+ * of filters.
+ * @param n - max number of records
+ * @param filters - list of filters
+ * @return backup records
+ * @throws IOException
+ */
+ public List getBackupHistory(int n, BackupInfo.Filter... filters) throws IOException {
+ if (filters.length == 0) return getHistory(n);
+
+ List history = 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;
+
+ }
+
+ /**
+ * Get history for backup destination
+ * @param backupRoot - backup destination
+ * @return List of backup info
+ * @throws IOException
+ */
+ public List getBackupHistory(String backupRoot) throws IOException {
+ ArrayList history = getBackupHistory(false);
+ for (Iterator iterator = history.iterator(); iterator.hasNext();) {
+ BackupInfo info = iterator.next();
+ if (!backupRoot.equals(info.getTargetRootDir())) {
+ iterator.remove();
+ }
+ }
+ return history;
+ }
+
+ /**
+ * Get history for a table
+ * @param name - table name
+ * @return history for a table
+ * @throws IOException
+ */
+ public List getBackupHistoryForTable(TableName name) throws IOException {
+ List history = getBackupHistory();
+ List tableHistory = new ArrayList();
+ for (BackupInfo info : history) {
+ List tables = info.getTableNames();
+ if (tables.contains(name)) {
+ tableHistory.add(info);
+ }
+ }
+ return tableHistory;
+ }
+
+ public Map>
+ getBackupHistoryForTableSet(Set set, String backupRoot) throws IOException {
+ List history = getBackupHistory(backupRoot);
+ Map> tableHistoryMap =
+ new HashMap>();
+ for (Iterator iterator = history.iterator(); iterator.hasNext();) {
+ BackupInfo info = iterator.next();
+ if (!backupRoot.equals(info.getTargetRootDir())) {
+ continue;
+ }
+ List tables = info.getTableNames();
+ for (TableName tableName: tables) {
+ if (set.contains(tableName)) {
+ ArrayList list = tableHistoryMap.get(tableName);
+ if (list == null) {
+ list = new ArrayList();
+ tableHistoryMap.put(tableName, list);
+ }
+ list.add(info);
+ }
+ }
+ }
+ return tableHistoryMap;
+ }
+
+ /**
+ * Get all backup session with a given status (in desc order by time)
+ * @param status status
+ * @return history info of backup contexts
+ * @throws IOException exception
+ */
+ public ArrayList getBackupContexts(BackupState status) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("get backup contexts from hbase:backup");
+ }
+
+ Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+ ArrayList list = new ArrayList();
+
+ try (Table table = connection.getTable(tableName);
+ ResultScanner scanner = table.getScanner(scan)) {
+ Result res = null;
+ while ((res = scanner.next()) != null) {
+ res.advance();
+ BackupInfo context = BackupSystemTableHelper.cellToBackupInfo(res.current());
+ if (status != BackupState.ANY && context.getState() != status) {
+ continue;
+ }
+ list.add(context);
+ }
+ return list;
+ }
+ }
+
+ /**
+ * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+ * incremental backup. The saved timestamp is of the last log file that was backed up already.
+ * @param tables tables
+ * @param newTimestamps timestamps
+ * @param backupRoot root directory path to backup
+ * @throws IOException exception
+ */
+ public void writeRegionServerLogTimestamp(Set tables,
+ HashMap newTimestamps, String backupRoot) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("write RS log time stamps to hbase:backup for tables ["
+ + StringUtils.join(tables, ",") + "]");
+ }
+ List puts = new ArrayList();
+ for (TableName table : tables) {
+ byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
+ Put put =
+ BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smapData,
+ backupRoot);
+ puts.add(put);
+ }
+ try (Table table = connection.getTable(tableName)) {
+ table.put(puts);
+ }
+ }
+
+ /**
+ * Read the timestamp for each region server log after the last successful backup. Each table has
+ * its own set of the timestamps. The info is stored for each table as a concatenated string of
+ * rs->timestapmp
+ * @param backupRoot root directory path to backup
+ * @return the timestamp for each region server. key: tableName value:
+ * RegionServer,PreviousTimeStamp
+ * @throws IOException exception
+ */
+ public HashMap> readLogTimestampMap(String backupRoot)
+ throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("read RS log ts from hbase:backup for root=" + backupRoot);
+ }
+
+ HashMap> tableTimestampMap =
+ new HashMap