hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e...@apache.org
Subject [47/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)
Date Tue, 22 Mar 2016 02:42:16 GMT
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
new file mode 100644
index 0000000..7bd6e99
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
@@ -0,0 +1,702 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+/**
+ * A Handler to carry the operations of backup progress
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupHandler implements Callable<Void> {
+  private static final Log LOG = LogFactory.getLog(BackupHandler.class);
+
+  // backup phase
+  // for overall backup (for table list, some table may go online, while some may go offline)
+  protected static enum BackupPhase {
+    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  // backup status flag
+  public static enum BackupState {
+    WAITING, RUNNING, COMPLETE, FAILED, CANCELLED;
+  }
+
+  protected final BackupContext backupContext;
+  private final BackupManager backupManager;
+  private final Configuration conf;
+  private final Connection conn;
+
+  public BackupHandler(BackupContext backupContext,
+      BackupManager backupManager, Configuration conf, Connection connection) {
+    this.backupContext = backupContext;
+    this.backupManager = backupManager;
+    this.conf = conf;
+    this.conn = connection;
+  }
+
+  public BackupContext getBackupContext() {
+    return backupContext;
+  }
+
+  @Override
+  public Void call() throws Exception {
+    try(Admin admin = conn.getAdmin()) {
+      // overall backup begin
+      this.beginBackup(backupContext);
+      HashMap<String, Long> newTimestamps = null;
+      // handle full or incremental backup for table or table list
+      if (backupContext.getType() == BackupType.FULL) {
+        String savedStartCode = null;
+        boolean firstBackup = false;
+        // do snapshot for full table backup
+
+        try {
+          savedStartCode = backupManager.readBackupStartCode();
+          firstBackup = savedStartCode == null;
+          if (firstBackup) {
+            // This is our first backup. Let's put some marker on ZK so that we can hold the logs
+            // while we do the backup.
+            backupManager.writeBackupStartCode(0L);
+          }
+          // We roll log here before we do the snapshot. It is possible there is duplicate data
+          // in the log that is already in the snapshot. But if we do it after the snapshot, we
+          // could have data loss.
+          // A better approach is to do the roll log on each RS in the same global procedure as
+          // the snapshot.
+          LOG.info("Execute roll log procedure for full backup ...");
+          admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+            LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
+          newTimestamps = backupManager.readRegionServerLastLogRollResult();
+          if (firstBackup) {
+            // Updates registered log files
+            // We record ALL old WAL files as registered, because
+            // this is a first full backup in the system and these
+            // files are not needed for next incremental backup
+            List<String> logFiles = BackupUtil.getWALFilesOlderThan(conf, newTimestamps);
+            backupManager.recordWALFiles(logFiles);
+          }
+          this.snapshotForFullBackup(backupContext);
+        } catch (BackupException e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected BackupException : ");
+          return null;
+        }
+
+        // update the faked progress currently for snapshot done
+        updateProgress(backupContext, backupManager, 10, 0);
+        // do snapshot copy
+        try {
+          this.snapshotCopy(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected BackupException : ");
+          return null;
+        }
+        // Updates incremental backup table set
+        backupManager.addIncrementalBackupTableSet(backupContext.getTables());
+
+      } else if (backupContext.getType() == BackupType.INCREMENTAL) {
+        LOG.debug("For incremental backup, current table set is "
+            + backupManager.getIncrementalBackupTableSet());
+        // do incremental table backup preparation
+        backupContext.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+        // avoid action if has been cancelled
+        if (backupContext.isCancelled()) {
+          return null;
+        }
+        try {
+          IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager);
+
+          newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected Exception : ");
+          return null;
+        }
+        // update the faked progress currently for incremental preparation done
+        updateProgress(backupContext, backupManager, 10, 0);
+
+        // do incremental copy
+        try {
+          // copy out the table and region info files for each table
+          BackupUtil.copyTableRegionInfo(backupContext, conf);
+          this.incrementalCopy(backupContext);
+          // Save list of WAL files copied
+          backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected exception doing incremental copy : ");
+          return null;
+        }
+      }
+
+      // set overall backup status: complete. Here we make sure to complete the backup. After this
+      // checkpoint, even if entering cancel process, will let the backup finished
+      backupContext.setState(BackupState.COMPLETE);
+
+      if (backupContext.getType() == BackupType.INCREMENTAL) {
+        // Set the previousTimestampMap which is before this current log roll to the manifest.
+        HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
+            backupManager.readLogTimestampMap();
+        backupContext.setIncrTimestampMap(previousTimestampMap);
+      }
+
+      // The table list in backupContext is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupUtility.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+
+      // backup complete
+      this.completeBackup(backupContext);
+    } catch (Exception e) {
+      // even during completing backup (#completeBackup(backupContext)), exception may occur, or
+      // exception occur during other process, fail the backup finally
+      this.failBackup(backupContext, e, "Error caught during backup progress: ");
+    }
+    return null;
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void beginBackup(BackupContext backupContext) throws IOException {
+    // set the start timestamp of the overall backup
+    long startTs = EnvironmentEdgeManager.currentTime();
+    backupContext.setStartTs(startTs);
+    // set overall backup status: ongoing
+    backupContext.setState(BackupState.RUNNING);
+    LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + ".");
+
+    backupManager.updateBackupStatus(backupContext);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
+    }
+  }
+
+  /**
+   * Snapshot for full table backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void snapshotForFullBackup(BackupContext backupContext) throws IOException {
+    LOG.info("HBase snapshot full backup for " + backupContext.getBackupId());
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    try (Admin admin = conn.getAdmin()) {
+      // we do HBase snapshot for tables in the table list one by one currently
+      for (TableName table : backupContext.getTables()) {
+        // avoid action if it has been cancelled
+        if (backupContext.isCancelled()) {
+          return;
+        }
+
+        HBaseProtos.SnapshotDescription backupSnapshot;
+
+        // wrap a SnapshotDescription for offline/online snapshot
+        backupSnapshot = this.wrapSnapshotDescription(table);
+
+        try {
+          // Kick off snapshot for backup
+          admin.snapshot(backupSnapshot);
+        } catch (Exception e) {
+          LOG.error("Snapshot failed to create " + getMessage(e));
+
+          // currently, we fail the overall backup if any table in the list failed, so throw the
+          // exception out for overall backup failing
+          throw new BackupException("Backup snapshot failed on table " + table, e);
+        }
+
+        // set the snapshot name in BackupStatus of this table, only after snapshot success.
+        backupContext.setSnapshotName(table, backupSnapshot.getName());
+      }
+    }
+  }
+
+  /**
+   * Fail the overall backup.
+   * @param backupContext backup context
+   * @param e exception
+   * @throws Exception exception
+   */
+  private void failBackup(BackupContext backupContext, Exception e, String msg) throws Exception {
+    LOG.error(msg + getMessage(e));
+    // If this is a cancel exception, then we've already cleaned.
+
+    if (this.backupContext.getState().equals(BackupState.CANCELLED)) {
+      return;
+    }
+
+    // set the failure timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+
+    // set failure message
+    backupContext.setFailedMsg(e.getMessage());
+
+    // set overall backup status: failed
+    backupContext.setState(BackupState.FAILED);
+
+    // compose the backup failed data
+    String backupFailedData =
+        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
+        + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
+        + ",failedmessage=" + backupContext.getFailedMsg();
+    LOG.error(backupFailedData);
+
+    backupManager.updateBackupStatus(backupContext);
+
+    // if full backup, then delete HBase snapshots if there already have snapshots taken
+    // and also clean up export snapshot log files if exist
+    if (backupContext.getType() == BackupType.FULL) {
+      this.deleteSnapshot(backupContext);
+      this.cleanupExportSnapshotLog();
+    } /*
+     * else { // support incremental backup code in future jira // TODO. See HBASE-14124 }
+     */
+
+    // clean up the uncompleted data at target directory if the ongoing backup has already entered
+    // the copy phase
+    // For incremental backup, DistCp logs will be cleaned with the targetDir.
+    this.cleanupTargetDir();
+
+    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
+  }
+
+  /**
+   * Update the ongoing back token znode with new progress.
+   * @param backupContext backup context
+   * 
+   * @param newProgress progress
+   * @param bytesCopied bytes copied
+   * @throws NoNodeException exception
+   */
+  public static void updateProgress(BackupContext backupContext, BackupManager backupManager,
+      int newProgress, long bytesCopied) throws IOException {
+    // compose the new backup progress data, using fake number for now
+    String backupProgressData = newProgress + "%";
+
+    backupContext.setProgress(newProgress);
+    backupManager.updateBackupStatus(backupContext);
+    LOG.debug("Backup progress data \"" + backupProgressData
+      + "\" has been updated to hbase:backup for " + backupContext.getBackupId());
+  }
+
+  /**
+   * Complete the overall backup.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void completeBackup(BackupContext backupContext) throws Exception {
+
+    // set the complete timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+    // set overall backup status: complete
+    backupContext.setState(BackupState.COMPLETE);
+    // add and store the manifest for the backup
+    this.addManifest(backupContext);
+
+    // after major steps done and manifest persisted, do convert if needed for incremental backup
+    /* in-fly convert code here, provided by future jira */
+    LOG.debug("in-fly convert code here, provided by future jira");
+
+    // compose the backup complete data
+    String backupCompleteData =
+        this.obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
+        + ",completets=" + backupContext.getEndTs() + ",bytescopied="
+        + backupContext.getTotalBytesCopied();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
+    }
+    backupManager.updateBackupStatus(backupContext);
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (backupContext.getType() == BackupType.FULL) {
+      this.deleteSnapshot(backupContext);
+      this.cleanupExportSnapshotLog();
+    } else if (backupContext.getType() == BackupType.INCREMENTAL) {
+      this.cleanupDistCpLog();
+    }
+
+    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
+  }
+
+  /**
+   * Get backup request meta data dir as string.
+   * @param backupContext backup context
+   * @return meta data dir
+   */
+  private String obtainBackupMetaDataStr(BackupContext backupContext) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("type=" + backupContext.getType() + ",tablelist=");
+    for (TableName table : backupContext.getTables()) {
+      sb.append(table + ";");
+    }
+    if (sb.lastIndexOf(";") > 0) {
+      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
+    }
+    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
+
+    return sb.toString();
+  }
+
+  /**
+   * Do snapshot copy.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void snapshotCopy(BackupContext backupContext) throws Exception {
+    LOG.info("Snapshot copy is starting.");
+
+    // set overall backup phase: snapshot_copy
+    backupContext.setPhase(BackupPhase.SNAPSHOTCOPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // call ExportSnapshot to copy files based on hbase snapshot for backup
+    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
+    BackupCopyService copyService = BackupRestoreFactory.getBackupCopyService(conf);
+
+    // number of snapshots matches number of tables
+    float numOfSnapshots = backupContext.getSnapshotNames().size();
+
+    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
+
+    for (TableName table : backupContext.getTables()) {
+      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
+      // calculate the real files' size for the percentage in the future.
+      // TODO this below
+      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
+      int res = 0;
+      String[] args = new String[4];
+      args[0] = "-snapshot";
+      args[1] = backupContext.getSnapshotName(table);
+      args[2] = "-copy-to";
+      args[3] = backupContext.getBackupStatus(table).getTargetDir();
+
+      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
+      res = copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.FULL, args);
+      // if one snapshot export failed, do not continue for remained snapshots
+      if (res != 0) {
+        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
+
+        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
+            + " with reason code " + res);
+      }
+
+      LOG.info("Snapshot copy " + args[1] + " finished.");
+    }
+  }
+
+  /**
+   * Wrap a SnapshotDescription for a target table.
+   * @param table table
+   * @return a SnapshotDescription especially for backup.
+   */
+  private SnapshotDescription wrapSnapshotDescription(TableName tableName) {
+    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
+    // Name it in the format "snapshot_<timestamp>_<table>"
+    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
+    builder.setTable(tableName.getNameAsString());
+    builder.setName("snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+        + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString());
+    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
+
+    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
+      + " from backupContext to request snapshot for backup.");
+
+    return backupSnapshot;
+  }
+
+  /**
+   * Delete HBase snapshot for backup.
+   * @param backupCtx backup context
+   * @throws Exception exception
+   */
+  private void deleteSnapshot(BackupContext backupCtx) throws IOException {
+
+    LOG.debug("Trying to delete snapshot for full backup.");
+    Connection conn = null;
+    Admin admin = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      admin = conn.getAdmin();
+      for (String snapshotName : backupCtx.getSnapshotNames()) {
+        if (snapshotName == null) {
+          continue;
+        }
+        LOG.debug("Trying to delete snapshot: " + snapshotName);
+        admin.deleteSnapshot(snapshotName);
+        LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
+            + backupCtx.getBackupId() + " succeeded.");
+      }
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
+   * snapshots.
+   * @throws IOException exception
+   */
+  private void cleanupExportSnapshotLog() throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+    Path stagingDir =
+        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
+          .toString()));
+    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("exportSnapshot-")) {
+        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
+        if (FSUtils.delete(fs, file.getPath(), true) == false) {
+          LOG.warn("Can not delete " + file.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
+   * hlogs.
+   * @throws IOException exception
+   */
+  private void cleanupDistCpLog() throws IOException {
+    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("_distcp_logs")) {
+        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
+        FSUtils.delete(fs, file.getPath(), true);
+      }
+    }
+  }
+
+  /**
+   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
+   * copy phase.
+   */
+  private void cleanupTargetDir() {
+    try {
+      // clean up the uncompleted data at target directory if the ongoing backup has already entered
+      // the copy phase
+      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
+          + backupContext.getPhase());
+      if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
+          || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
+          || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
+        FileSystem outputFs =
+            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
+
+        // now treat one backup as a transaction, clean up data that has been partially copied at
+        // table level
+        for (TableName table : backupContext.getTables()) {
+          Path targetDirPath =
+              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
+                backupContext.getBackupId(), table));
+          if (outputFs.delete(targetDirPath, true)) {
+            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+              + " done.");
+          } else {
+            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
+          }
+
+          Path tableDir = targetDirPath.getParent();
+          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
+          if (backups == null || backups.length == 0) {
+            outputFs.delete(tableDir, true);
+            LOG.debug(tableDir.toString() + " is empty, remove it.");
+          }
+        }
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
+          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Add manifest for the current backup. The manifest is stored
+   * within the table backup directory.
+   * @param backupContext The current backup context
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  private void addManifest(BackupContext backupContext) throws IOException, BackupException {
+    // set the overall backup phase : store manifest
+    backupContext.setPhase(BackupPhase.STORE_MANIFEST);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    BackupManifest manifest;
+
+    // Since we have each table's backup in its own directory structure,
+    // we'll store its manifest with the table directory.
+    for (TableName table : backupContext.getTables()) {
+      manifest = new BackupManifest(backupContext, table);
+      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext, table);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+
+      if (backupContext.getType() == BackupType.INCREMENTAL) {
+        // We'll store the log timestamps for this table only in its manifest.
+        HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+            new HashMap<TableName, HashMap<String, Long>>();
+        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
+        manifest.setIncrTimestampMap(tableTimestampMap);
+      }
+      manifest.store(conf);
+    }
+
+    // For incremental backup, we store a overall manifest in
+    // <backup-root-dir>/WALs/<backup-id>
+    // This is used when created the next incremental backup
+    if (backupContext.getType() == BackupType.INCREMENTAL) {
+      manifest = new BackupManifest(backupContext);
+      // set the table region server start and end timestamps for incremental backup
+      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
+      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+      manifest.store(conf);
+    }
+  }
+
+  /**
+   * Do incremental copy.
+   * @param backupContext backup context
+   */
+  private void incrementalCopy(BackupContext backupContext) throws Exception {
+
+    LOG.info("Incremental copy is starting.");
+
+    // set overall backup phase: incremental_copy
+    backupContext.setPhase(BackupPhase.INCREMENTAL_COPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // get incremental backup file list and prepare parms for DistCp
+    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
+
+    BackupCopyService copyService = BackupRestoreFactory.getBackupCopyService(conf);
+    int res = copyService.copy(backupContext, backupManager, conf,
+      BackupCopyService.Type.INCREMENTAL, strArr);
+
+    if (res != 0) {
+      LOG.error("Copy incremental log files failed with return code: " + res + ".");
+      throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to "
+          + backupContext.getHLogTargetDir());
+    }
+    LOG.info("Incremental copy from " + incrBackupFileList + " to "
+        + backupContext.getHLogTargetDir() + " finished.");
+
+  }
+
+  private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for(String file : incrBackupFileList){
+      if(fs.exists(new Path(file))){
+        list.add(file);
+      } else{
+        LOG.warn("Can't find file: "+file);
+      }
+    }
+    return list;
+  }
+
+  private String getMessage(Exception e) {
+    String msg = e.getMessage();
+    if (msg == null || msg.equals("")) {
+      msg = e.getClass().getName();
+    }
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..a4b0a0a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,512 @@
+/**
+ * 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 com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+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.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+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.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+/**
+ * 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 BackupContext backupContext = null;
+
+  private ExecutorService pool = null;
+
+  private boolean backupComplete = false;
+
+  private BackupSystemTable systemTable;
+
+  private final Connection conn;
+
+  /**
+   * Backup manager constructor.
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public BackupManager(Configuration conf) throws IOException {
+    if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
+      throw new BackupException("HBase backup is not enabled. Check your " +
+          HConstants.BACKUP_ENABLE_KEY + " setting.");
+    }
+    this.conf = conf;
+    this.conn = ConnectionFactory.createConnection(conf); // TODO: get Connection from elsewhere?
+    this.systemTable = new BackupSystemTable(conn);
+    Runtime.getRuntime().addShutdownHook(new ExitHandler());
+  }
+
+  /**
+   * 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;
+    }
+    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);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Added log cleaner: " + cleanerClass);
+    }
+  }
+
+  private static boolean isBackupEnabled(Configuration conf) {
+    return conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT);
+  }
+
+  // TODO: remove this on the server side
+  private class ExitHandler extends Thread {
+    public ExitHandler() {
+      super("Backup Manager Exit Handler");
+    }
+
+    @Override
+    public void run() {
+      if (backupContext != null && !backupComplete) {
+
+        // program exit and backup is not complete, then mark as cancelled to avoid submitted backup
+        // handler's taking further action
+        backupContext.setCancelled(true);
+
+        LOG.debug("Backup is cancelled due to force program exiting.");
+        try {
+          cancelBackup(backupContext.getBackupId());
+        } catch (Exception e) {
+          String msg = e.getMessage();
+          if (msg == null || msg.equals("")) {
+            msg = e.getClass().getName();
+          }
+          LOG.error("Failed to cancel backup " + backupContext.getBackupId() + " due to " + msg);
+        }
+      }
+      close();
+    }
+  }
+
+  /**
+   * Get configuration
+   * @return configuration
+   */
+  Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Cancel the ongoing backup via backup id.
+   * @param backupId The id of the ongoing backup to be cancelled
+   * @throws Exception exception
+   */
+  private void cancelBackup(String backupId) throws Exception {
+    // TODO: will be implemented in Phase 2: HBASE-14125
+    LOG.debug("Try to cancel the backup " + backupId + ". the feature is NOT implemented yet");
+
+  }
+
+  /**
+   * 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 snapshot snapshot name
+   * @return BackupContext context
+   * @throws BackupException exception
+   */
+  protected BackupContext createBackupContext(String backupId, BackupType type,
+      List<TableName> tableList, String targetRootDir) throws BackupException {
+
+    if (targetRootDir == null) {
+      throw new BackupException("Wrong backup request parameter: target backup root directory");
+    }
+
+    if (type == BackupType.FULL && tableList == null) {
+      // 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 hbadmin = conn.getAdmin()) {
+        htds = hbadmin.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
+    return new BackupContext(backupId, type, tableList.toArray(new TableName[tableList.size()]),
+      targetRootDir);
+  }
+
+  /**
+   * 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<BackupContext> 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<Runnable>(), builder.build());
+    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * Dispatch and handle a backup request.
+   * @param backupContext backup context
+   * @throws BackupException exception
+   */
+  public void dispatchRequest(BackupContext backupContext) throws BackupException {
+
+    this.backupContext = backupContext;
+
+    LOG.info("Got a backup request: " + "Type: " + backupContext.getType() + "; Tables: "
+        + backupContext.getTableNames() + "; TargetRootDir: " + backupContext.getTargetRootDir());
+
+    // dispatch the request to a backup handler and put it handler map
+
+    BackupHandler handler = new BackupHandler(this.backupContext, this, conf, this.conn);
+    Future<Void> future = this.pool.submit(handler);
+    // wait for the execution to complete
+    try {
+      future.get();
+    } catch (InterruptedException e) {
+      throw new BackupException(e);
+    } catch (CancellationException e) {
+      throw new BackupException(e);
+    } catch (ExecutionException e) {
+      throw new BackupException(e);
+    }
+
+    // mark the backup complete for exit handler's processing
+    backupComplete = true;
+
+    LOG.info("Backup request " + backupContext.getBackupId() + " has been executed.");
+  }
+
+  /**
+   * 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
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupCtx) throws IOException,
+      BackupException {
+    LOG.debug("Getting the direct ancestors of the current backup ...");
+
+    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
+
+    // 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<BackupCompleteData> allHistoryList = getBackupHistory();
+    for (BackupCompleteData backup : allHistoryList) {
+      BackupImage image =
+          new BackupImage(backup.getBackupToken(), BackupType.valueOf(backup.getType()),
+            backup.getBackupRootPath(),
+              backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup
+                  .getEndTime()));
+      // add the full backup image as an ancestor until the last incremental backup
+      if (backup.getType().equals(BackupType.FULL.toString())) {
+        // 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.getBackupRootPath(),
+                backup.getBackupToken());
+          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
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupContext, TableName table)
+      throws BackupException, IOException {
+    ArrayList<BackupImage> ancestors = getAncestors(backupContext);
+    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
+    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 updateBackupStatus(BackupContext context) throws IOException {
+    systemTable.updateBackupStatus(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();
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * Get the RS log information after the last log roll from hbase:backup.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult() throws IOException {
+    return systemTable.readRegionServerLastLogRollResult();
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
+    return systemTable.getBackupHistory();
+  }
+
+  /**
+   * 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<TableName> tables,
+      HashMap<String, Long> newTimestamps) throws IOException {
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps);
+  }
+
+  /**
+   * 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<TableName, HashMap<String, Long>> readLogTimestampMap() throws IOException {
+    return systemTable.readLogTimestampMap();
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet() throws IOException {
+    return systemTable.getIncrementalBackupTableSet();
+  }
+
+  /**
+   * Adds set of tables to overall incremental backup table set
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOException {
+    systemTable.addIncrementalBackupTableSet(tables);
+  }
+
+  /**
+   * 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<String> files) throws IOException {
+    systemTable.addWALFiles(files, backupContext.getBackupId());
+  }
+
+  /**
+   * Get WAL files iterator
+   * @return WAL files iterator from hbase:backup
+   * @throws IOException
+   */
+  public Iterator<String> getWALFilesFromBackupSystem() throws IOException {
+    return  systemTable.getWALFilesIterator();
+  }
+
+  public Connection getConnection() {
+    return conn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..6264fc5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,762 @@
+/**
+ * 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.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+
+/**
+ * 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";
+
+  // manifest file version, current is 1.0
+  public static final String MANIFEST_VERSION = "1.0";
+
+  // backup image, the dependency graph is made up by series of backup images
+
+  public static class BackupImage implements Comparable<BackupImage> {
+
+    private String backupId;
+    private BackupType type;
+    private String rootDir;
+    private List<TableName> tableList;
+    private long startTs;
+    private long completeTs;
+    private ArrayList<BackupImage> ancestors;
+
+    public BackupImage() {
+      super();
+    }
+
+    public BackupImage(String backupId, BackupType type, String rootDir,
+        List<TableName> 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<HBaseProtos.TableName> tableListList = im.getTableListList();
+      List<TableName> tableList = new ArrayList<TableName>();
+      for(HBaseProtos.TableName tn : tableListList) {
+        tableList.add(ProtobufUtil.toTableName(tn));
+      }
+      BackupType type =
+          im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL:
+            BackupType.INCREMENTAL;
+
+      return new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
+    }
+
+    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.toProtoTableName(name));
+      }
+
+      if (ancestors != null){
+        for (BackupImage im: ancestors){
+          builder.addAncestors(im.toProto());
+        }
+      }
+
+      return builder.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<TableName> getTableNames() {
+      return tableList;
+    }
+
+    public void setTableList(List<TableName> 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<BackupImage> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<BackupImage>();
+      }
+      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.getNameAsString().equals(table)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public int compareTo(BackupImage other) {
+      String thisBackupId = this.getBackupId();
+      String otherBackupId = other.getBackupId();
+      Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1));
+      Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1));
+      return thisTS.compareTo(otherTS);
+    }
+  }
+
+  // manifest version
+  private String version = MANIFEST_VERSION;
+
+  // 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<TableName> tableList;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual complete timestamp of the backup process
+  private long completeTs;
+
+  // total bytes for table backup image
+  private long totalBytes;
+
+  // total bytes for the backed-up logs for incremental backup
+  private long logBytes;
+
+  // the region server timestamp for tables:
+  // <table, <rs, timestamp>>
+  private Map<TableName, HashMap<String, Long>> incrTimeRanges;
+
+  // dependency of this backup, including all the dependent images to do PIT recovery
+  private Map<String, BackupImage> dependency;
+
+  // the indicator of the image compaction
+  private boolean isCompacted = false;
+  /**
+   * Construct manifest for a ongoing backup.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext backupCtx) {
+    this.backupId = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    if (this.type == BackupType.INCREMENTAL) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+      this.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    this.loadTableList(backupCtx.getTableNames());
+  }
+
+  /**
+   * Construct a table level manifest for a backup of the named table.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext 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.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    List<TableName> tables = new ArrayList<TableName>();
+    tables.add(table);
+    this.loadTableList(tables);
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws BackupException exception
+   */
+
+  public BackupManifest(Configuration conf, 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 = conf;
+    try {
+
+      FileSystem fs = backupPath.getFileSystem(conf);
+      FileStatus[] subFiles = FSUtils.listStatus(fs, backupPath);
+      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.BackupManifest proto = null;
+          try{
+            proto = parseFrom(pbBytes);
+          } catch(Exception e){
+            throw new BackupException(e);
+          }
+          this.version = proto.getVersion();
+          this.backupId = proto.getBackupId();
+          this.type = BackupType.valueOf(proto.getType().name());
+          // 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();
+          }
+
+          loadTableList(proto);
+          this.startTs = proto.getStartTs();
+          this.completeTs = proto.getCompleteTs();
+          this.totalBytes = proto.getTotalBytes();
+          if (this.type == BackupType.INCREMENTAL) {
+            this.logBytes = proto.getLogBytes();
+            //TODO: convert will be implemented by future jira
+          }
+
+          loadIncrementalTimestampMap(proto);
+          loadDependency(proto);
+          this.isCompacted = proto.getCompacted();
+          //TODO: merge will be implemented by future jira
+          LOG.debug("Loaded manifest instance from manifest file: "
+              + FSUtils.getPath(subFile.getPath()));
+          return;
+        }
+      }
+      String errorMsg = "No manifest file found in: " + backupPath.toString();
+      LOG.error(errorMsg);
+      throw new IOException(errorMsg);
+
+    } catch (IOException e) {
+      LOG.error(e);
+      throw new BackupException(e.getMessage());
+    }
+  }
+
+  private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
+    List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
+    if(list == null || list.size() == 0) return;
+    this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    for(BackupProtos.TableServerTimestamp tst: list){
+      TableName tn = ProtobufUtil.toTableName(tst.getTable());
+      HashMap<String, Long> map = this.incrTimeRanges.get(tn);
+      if(map == null){
+        map = new HashMap<String, Long>();
+        this.incrTimeRanges.put(tn, map);
+      }
+      List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
+      for(BackupProtos.ServerTimestamp stm: listSt) {
+        map.put(stm.getServer(), stm.getTimestamp());
+      }
+    }
+  }
+
+  private void loadDependency(BackupProtos.BackupManifest proto) {
+    dependency = new HashMap<String, BackupImage>();
+    List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
+    for (BackupProtos.BackupImage im : list) {
+      dependency.put(im.getBackupId(), BackupImage.fromProto(im));
+    }
+  }
+
+  private void loadTableList(BackupProtos.BackupManifest proto) {
+    this.tableList = new ArrayList<TableName>();
+    List<HBaseProtos.TableName> list = proto.getTableListList();
+    for (HBaseProtos.TableName name: list) {
+      this.tableList.add(ProtobufUtil.toTableName(name));
+    }
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  /**
+   * Loads table list.
+   * @param tableList Table list
+   */
+  private void loadTableList(List<TableName> 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<TableName> getTableList() {
+    if (this.tableList == null) {
+      this.tableList = new ArrayList<TableName>();
+    }
+    return this.tableList;
+  }
+
+  /**
+   * Persist the manifest file.
+   * @throws IOException IOException when storing the manifest file.
+   */
+
+  public void store(Configuration conf) throws BackupException {
+    byte[] data = 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) {
+      LOG.error(e);
+      throw new BackupException(e.getMessage());
+    }
+
+    LOG.debug("Manifestfilestored_to " + this.tableBackupDir != null ? this.tableBackupDir
+        : this.logBackupDir + Path.SEPARATOR + MANIFEST_FILE_NAME);
+  }
+
+  /**
+   * Protobuf serialization
+   * @return The filter serialized using pb
+   */
+  public byte[] toByteArray() {
+    BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
+    builder.setVersion(this.version);
+    builder.setBackupId(this.backupId);
+    builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
+    setTableList(builder);
+    builder.setStartTs(this.startTs);
+    builder.setCompleteTs(this.completeTs);
+    builder.setTotalBytes(this.totalBytes);
+    if (this.type == BackupType.INCREMENTAL) {
+      builder.setLogBytes(this.logBytes);
+    }
+    setIncrementalTimestampMap(builder);
+    setDependencyMap(builder);
+    builder.setCompacted(this.isCompacted);
+    return builder.build().toByteArray();
+  }
+
+  private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
+    if (this.incrTimeRanges == null) return;
+    for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
+      TableName key = entry.getKey();
+      HashMap<String, Long> value = entry.getValue();
+      BackupProtos.TableServerTimestamp.Builder tstBuilder =
+          BackupProtos.TableServerTimestamp.newBuilder();
+      tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
+
+      for (String s : value.keySet()) {
+        BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
+        stBuilder.setServer(s);
+        stBuilder.setTimestamp(value.get(s));
+        tstBuilder.addServerTimestamp(stBuilder.build());
+      }
+      builder.addTstMap(tstBuilder.build());
+    }
+  }
+
+  private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
+    for (BackupImage image: getDependency().values()) {
+      builder.addDependentBackupImage(image.toProto());
+    }
+  }
+
+  private void setTableList(BackupProtos.BackupManifest.Builder builder) {
+    for(TableName name: tableList){
+      builder.addTableList(ProtobufUtil.toProtoTableName(name));
+    }
+  }
+
+  /**
+   * Parse protobuf from byte array
+   * @param pbBytes A pb serialized BackupManifest instance
+   * @return An instance of  made from <code>bytes</code>
+   * @throws DeserializationException
+   */
+  private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
+      throws DeserializationException {
+    BackupProtos.BackupManifest proto;
+    try {
+      proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return proto;
+  }
+
+  /**
+   * Get manifest file version
+   * @return version
+   */
+  public String getVersion() {
+    return version;
+  }
+
+  /**
+   * Get this backup image.
+   * @return the backup image.
+   */
+  public BackupImage getBackupImage() {
+    return this.getDependency().get(this.backupId);
+  }
+
+  /**
+   * Add dependent backup image for this backup.
+   * @param image The direct dependent backup image
+   */
+  public void addDependentImage(BackupImage image) {
+    this.getDependency().get(this.backupId).addAncestor(image);
+    this.setDependencyMap(this.getDependency(), image);
+  }
+
+
+
+  /**
+   * Get all dependent backup images. The image of this backup is also contained.
+   * @return The dependent backup images map
+   */
+  public Map<String, BackupImage> getDependency() {
+    if (this.dependency == null) {
+      this.dependency = new HashMap<String, BackupImage>();
+      LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
+      this.dependency.put(this.backupId,
+        new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
+            this.completeTs));
+    }
+    return this.dependency;
+  }
+
+  /**
+   * Set the incremental timestamp map directly.
+   * @param incrTimestampMap timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
+    this.incrTimeRanges = incrTimestampMap;
+  }
+
+
+  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    if (this.incrTimeRanges == null) {
+      this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    }
+    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<BackupImage> getRestoreDependentList(boolean reverse) {
+    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
+    for (BackupImage image : this.getDependency().values()) {
+      restoreImages.put(Long.valueOf(image.startTs), image);
+    }
+    return new ArrayList<BackupImage>(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<BackupImage> getDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> 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<BackupImage> getAllDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+      }
+    }
+    return tableImageList;
+  }
+
+
+  /**
+   * Recursively set the dependency map of the backup images.
+   * @param map The dependency map
+   * @param image The backup image
+   */
+  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
+    if (image == null) {
+      return;
+    } else {
+      map.put(image.getBackupId(), image);
+      for (BackupImage img : image.getAncestors()) {
+        setDependencyMap(map, img);
+      }
+    }
+  }
+
+  /**
+   * 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<TableName> image1TableList = image1.getTableNames();
+    List<TableName> 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<BackupImage> 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<String> image1TableList = new ArrayList<String>();
+    for (BackupImage image1 : fullImages) {
+      List<TableName> tableList = image1.getTableNames();
+      for (TableName table : tableList) {
+        image1TableList.add(table.getNameAsString());
+      }
+    }
+    ArrayList<String> image2TableList = new ArrayList<String>();
+    List<TableName> 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
new file mode 100644
index 0000000..d0ce059
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
@@ -0,0 +1,46 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * HConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public final class BackupRestoreConstants {
+
+
+  // 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,
+  }
+
+  private BackupRestoreConstants() {
+    // Can't be instantiated with this ctor.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
new file mode 100644
index 0000000..8b8a83f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
@@ -0,0 +1,42 @@
+/**
+ * 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 BackupHandler backupHandler;
+  private String table;
+
+  public BackupSnapshotCopy(BackupHandler backupHandler, String table) {
+    super();
+    this.backupHandler = backupHandler;
+    this.table = table;
+  }
+
+  public BackupHandler getBackupHandler() {
+    return this.backupHandler;
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
new file mode 100644
index 0000000..6e54994
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
@@ -0,0 +1,105 @@
+/**
+ * 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.Serializable;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+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.protobuf.generated.BackupProtos;
+
+/**
+ * Backup status and related information encapsulated for a table.
+ * At this moment only TargetDir and SnapshotName is encapsulated here.
+ * future Jira will be implemented for progress, bytesCopies, phase, etc.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupStatus implements Serializable {
+
+  private static final long serialVersionUID = -5968397963548535982L;
+
+  // table name for backup
+  private 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 = HBackupFileSystem.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.toProtoTableName(table));
+    builder.setTargetDir(targetDir);
+    return builder.build();
+  }
+}


Mime
View raw message