You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/08/23 16:47:24 UTC

[27/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
deleted file mode 100644
index 1765bf3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ /dev/null
@@ -1,550 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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.BackupUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.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.util.Bytes;
-
-
-/**
- * An object to encapsulate the information for each backup session
- */
-@InterfaceAudience.Private
-public class BackupInfo implements Comparable<BackupInfo> {
-  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 session states
-   */
-  public static enum BackupState {
-    RUNNING, COMPLETE, FAILED, ANY;
-  }
-
-  /**
-   * BackupPhase - phases of an ACTIVE backup session (running), when state of a backup session is
-   * BackupState.RUNNING
-   */
-  public static enum BackupPhase {
-    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, 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 backupRootDir;
-
-  /**
-   * Backup state
-   */
-  private BackupState state;
-
-  /**
-   * Backup phase
-   */
-  private BackupPhase phase = BackupPhase.REQUEST;
-
-  /**
-   * Backup failure message
-   */
-  private String failedMsg;
-
-  /**
-   * Backup status map for all tables
-   */
-  private Map<TableName, BackupTableInfo> backupTableInfoMap;
-
-  /**
-   * Actual start timestamp of a backup process
-   */
-  private long startTs;
-
-  /**
-   * Actual end timestamp of the backup process
-   */
-  private long completeTs;
-
-  /**
-   * 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
-   */
-  private List<String> 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
-   */
-  private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
-
-  /**
-   * Backup progress in %% (0-100)
-   */
-  private int progress;
-
-  /**
-   * 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() {
-    backupTableInfoMap = new HashMap<TableName, BackupTableInfo>();
-  }
-
-  public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
-    this();
-    this.backupId = backupId;
-    this.type = type;
-    this.backupRootDir = targetRootDir;
-    this.addTables(tables);
-    if (type == BackupType.INCREMENTAL) {
-      setHLogTargetDir(BackupUtils.getLogBackupDir(targetRootDir, backupId));
-    }
-    this.startTs = 0;
-    this.completeTs = 0;
-  }
-
-  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 setBackupTableInfoMap(Map<TableName, BackupTableInfo> backupTableInfoMap) {
-    this.backupTableInfoMap = backupTableInfoMap;
-  }
-
-  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
-    return tableSetTimestampMap;
-  }
-
-  public void setTableSetTimestampMap(HashMap<TableName,
-                                      HashMap<String, Long>> tableSetTimestampMap) {
-    this.tableSetTimestampMap = tableSetTimestampMap;
-  }
-
-  public void setType(BackupType type) {
-    this.type = type;
-  }
-
-  public void setBackupRootDir(String targetRootDir) {
-    this.backupRootDir = 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 BackupTableInfo getBackupTableInfo(TableName table) {
-    return this.backupTableInfoMap.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 getCompleteTs() {
-    return completeTs;
-  }
-
-  public void setCompleteTs(long endTs) {
-    this.completeTs = 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.backupTableInfoMap.get(table).setSnapshotName(snapshotName);
-  }
-
-  public String getSnapshotName(TableName table) {
-    return this.backupTableInfoMap.get(table).getSnapshotName();
-  }
-
-  public List<String> getSnapshotNames() {
-    List<String> snapshotNames = new ArrayList<String>();
-    for (BackupTableInfo backupStatus : this.backupTableInfoMap.values()) {
-      snapshotNames.add(backupStatus.getSnapshotName());
-    }
-    return snapshotNames;
-  }
-
-  public Set<TableName> getTables() {
-    return this.backupTableInfoMap.keySet();
-  }
-
-  public List<TableName> getTableNames() {
-    return new ArrayList<TableName>(backupTableInfoMap.keySet());
-  }
-
-  public void addTables(TableName[] tables) {
-    for (TableName table : tables) {
-      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
-      this.backupTableInfoMap.put(table, backupStatus);
-    }
-  }
-
-  public void setTables(List<TableName> tables) {
-    this.backupTableInfoMap.clear();
-    for (TableName table : tables) {
-      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
-      this.backupTableInfoMap.put(table, backupStatus);
-    }
-  }
-
-  public String getBackupRootDir() {
-    return backupRootDir;
-  }
-
-  public String getTableBackupDir(TableName tableName) {
-    return BackupUtils.getTableBackupDir(backupRootDir, backupId, tableName);
-  }
-
-  public void setHLogTargetDir(String hlogTagetDir) {
-    this.hlogTargetDir = hlogTagetDir;
-  }
-
-  public String getHLogTargetDir() {
-    return hlogTargetDir;
-  }
-
-  public List<String> getIncrBackupFileList() {
-    return incrBackupFileList;
-  }
-
-  public void setIncrBackupFileList(List<String> incrBackupFileList) {
-    this.incrBackupFileList = incrBackupFileList;
-  }
-
-  /**
-   * Set the new region server log timestamps after distributed log roll
-   * @param newTableSetTimestampMap table timestamp map
-   */
-  public void
-      setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap) {
-    this.tableSetTimestampMap = newTableSetTimestampMap;
-  }
-
-  /**
-   * Get new region server log timestamps after distributed log roll
-   * @return new region server log timestamps
-   */
-  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
-    return this.tableSetTimestampMap;
-  }
-
-  public TableName getTableBySnapshot(String snapshotName) {
-    for (Entry<TableName, BackupTableInfo> entry : this.backupTableInfoMap.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());
-    setBackupTableInfoMap(builder);
-    builder.setCompleteTs(getCompleteTs());
-    if (getFailedMsg() != null) {
-      builder.setFailedMessage(getFailedMsg());
-    }
-    if (getState() != null) {
-      builder.setBackupState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
-    }
-    if (getPhase() != null) {
-      builder.setBackupPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
-    }
-
-    builder.setProgress(getProgress());
-    builder.setStartTs(getStartTs());
-    builder.setBackupRootDir(getBackupRootDir());
-    builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name()));
-    builder.setWorkersNumber(workers);
-    builder.setBandwidth(bandwidth);
-    return builder.build();
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = 33 * type.hashCode() + backupId != null ? backupId.hashCode() : 0;
-    if (backupRootDir != null) {
-      hash = 33 * hash + backupRootDir.hashCode();
-    }
-    hash = 33 * hash + state.hashCode();
-    hash = 33 * hash + phase.hashCode();
-    hash = 33 * hash + (int) (startTs ^ (startTs >>> 32));
-    hash = 33 * hash + (int) (completeTs ^ (completeTs >>> 32));
-    hash = 33 * hash + (int) (totalBytesCopied ^ (totalBytesCopied >>> 32));
-    if (hlogTargetDir != null) {
-      hash = 33 * hash + hlogTargetDir.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;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return backupId;
-  }
-
-  public byte[] toByteArray() throws IOException {
-    return toProtosBackupInfo().toByteArray();
-  }
-
-  private void setBackupTableInfoMap(Builder builder) {
-    for (Entry<TableName, BackupTableInfo> entry : backupTableInfoMap.entrySet()) {
-      builder.addBackupTableInfo(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.setBackupTableInfoMap(toMap(proto.getBackupTableInfoList()));
-    context.setCompleteTs(proto.getCompleteTs());
-    if (proto.hasFailedMessage()) {
-      context.setFailedMsg(proto.getFailedMessage());
-    }
-    if (proto.hasBackupState()) {
-      context.setState(BackupInfo.BackupState.valueOf(proto.getBackupState().name()));
-    }
-
-    context.setHLogTargetDir(BackupUtils.getLogBackupDir(proto.getBackupRootDir(),
-      proto.getBackupId()));
-
-    if (proto.hasBackupPhase()) {
-      context.setPhase(BackupPhase.valueOf(proto.getBackupPhase().name()));
-    }
-    if (proto.hasProgress()) {
-      context.setProgress(proto.getProgress());
-    }
-    context.setStartTs(proto.getStartTs());
-    context.setBackupRootDir(proto.getBackupRootDir());
-    context.setType(BackupType.valueOf(proto.getBackupType().name()));
-    context.setWorkers(proto.getWorkersNumber());
-    context.setBandwidth(proto.getBandwidth());
-    return context;
-  }
-
-  private static Map<TableName, BackupTableInfo> toMap(List<BackupProtos.BackupTableInfo> list) {
-    HashMap<TableName, BackupTableInfo> map = new HashMap<>();
-    for (BackupProtos.BackupTableInfo tbs : list) {
-      map.put(ProtobufUtil.toTableName(tbs.getTableName()), BackupTableInfo.convert(tbs));
-    }
-    return map;
-  }
-
-  public String getShortDescription() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("{");
-    sb.append("ID=" + backupId).append(",");
-    sb.append("Type=" + getType()).append(",");
-    sb.append("Tables=" + getTableListAsString()).append(",");
-    sb.append("State=" + getState()).append(",");
-    Date date = null;
-    Calendar cal = Calendar.getInstance();
-    cal.setTimeInMillis(getStartTs());
-    date = cal.getTime();
-    sb.append("Start time=" + date).append(",");
-    if (state == BackupState.FAILED) {
-      sb.append("Failed message=" + getFailedMsg()).append(",");
-    } else if (state == BackupState.RUNNING) {
-      sb.append("Phase=" + getPhase()).append(",");
-    } else if (state == BackupState.COMPLETE) {
-      cal = Calendar.getInstance();
-      cal.setTimeInMillis(getCompleteTs());
-      date = cal.getTime();
-      sb.append("End time=" + date).append(",");
-    }
-    sb.append("Progress=" + getProgress() + "%");
-    sb.append("}");
-
-    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() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("{");
-    sb.append(StringUtils.join(backupTableInfoMap.keySet(), ","));
-    sb.append("}");
-    return sb.toString();
-  }
-
-  /**
-   * We use only time stamps to compare objects during sort operation
-   */
-  @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);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
deleted file mode 100644
index 136782f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
- * {@link BackupRestoreFactory}
- */
-
-@InterfaceAudience.Private
-public interface BackupMergeJob extends Configurable {
-
-  /**
-   * Run backup merge operation
-   * @param backupIds backup image ids
-   * @throws IOException
-   */
-  void run(String[] backupIds) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
deleted file mode 100644
index 595e862..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.impl.BackupManager;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * An Observer to facilitate backup operations
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupObserver implements RegionObserver {
-  private static final Log LOG = LogFactory.getLog(BackupObserver.class);
-  @Override
-  public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
-    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
-    boolean hasLoaded) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (!hasLoaded) {
-      // there is no need to record state
-      return hasLoaded;
-    }
-    if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
-      return hasLoaded;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return hasLoaded;
-      }
-      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
-      return hasLoaded;
-    } catch (IOException ioe) {
-      LOG.error("Failed to get tables which have been fully backed up", ioe);
-      return false;
-    }
-  }
-  @Override
-  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
-      return;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return;
-      }
-      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
-      return;
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
deleted file mode 100644
index 48e70a1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
- */
-@InterfaceAudience.Private
-public interface BackupRestoreConstants {
-  /*
-   * Backup/Restore constants
-   */
-  public final static String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name";
-  public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "backup:system";
-
-  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 = false;
-
-
-  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
-  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
-
-  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
-  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
-
-  /*
-   *  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 -t (table list)";
-  public static final String OPTION_SET_BACKUP_DESC =
-      "Backup set to backup, mutually exclusive with -t (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_TABLE_LIST = "l";
-  public static final String OPTION_TABLE_LIST_DESC = "Table name list, comma-separated.";
-
-  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 <tables> must have a mapping";
-
-  public static final String BACKUP_CONFIG_STRING =  BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n"
-      + "hbase.master.logcleaner.plugins="
-      +"YOUR_PLUGINS,org.apache.hadoop.hbase.backup.master.BackupLogCleaner\n"
-      + "hbase.procedure.master.classes=YOUR_CLASSES,"
-      +"org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager\n"
-      + "hbase.procedure.regionserver.classes=YOUR_CLASSES,"
-      + "org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager\n"
-      + "and restart the cluster";
-  public static final String ENABLE_BACKUP = "Backup is not enabled. To enable backup, "+
-      "in hbase-site.xml, set:\n "
-      + BACKUP_CONFIG_STRING;
-
-  public static final String VERIFY_BACKUP = "Please make sure that backup is enabled on the cluster. To enable backup, "+
-      "in hbase-site.xml, set:\n "
-      + BACKUP_CONFIG_STRING;
-
-  /*
-   *  Delimiter in table name 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, REPAIR
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
deleted file mode 100644
index d72c884..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * Factory implementation for backup/restore related jobs
- *
- */
-@InterfaceAudience.Private
-public final class BackupRestoreFactory {
-
-  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";
-  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
-
-  private BackupRestoreFactory() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Gets backup restore job
-   * @param conf configuration
-   * @return backup restore job instance
-   */
-  public static RestoreJob getRestoreJob(Configuration conf) {
-    Class<? extends RestoreJob> cls =
-        conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreJob.class, RestoreJob.class);
-    RestoreJob service = ReflectionUtils.newInstance(cls, conf);
-    service.setConf(conf);
-    return service;
-  }
-
-  /**
-   * Gets backup copy job
-   * @param conf configuration
-   * @return backup copy job instance
-   */
-  public static BackupCopyJob getBackupCopyJob(Configuration conf) {
-    Class<? extends BackupCopyJob> cls =
-        conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyJob.class,
-          BackupCopyJob.class);
-    BackupCopyJob service = ReflectionUtils.newInstance(cls, conf);
-    service.setConf(conf);
-    return service;
-  }
-
-  /**
-   * Gets backup merge job
-   * @param conf configuration
-   * @return backup merge job instance
-   */
-  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
-    Class<? extends BackupMergeJob> cls =
-        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
-          BackupMergeJob.class);
-    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
-    service.setConf(conf);
-    return service;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
deleted file mode 100644
index a5390a6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
-
-/**
- * Backup related information encapsulated for a table. At this moment only target directory,
- * snapshot name and table name are encapsulated here.
- */
-
-@InterfaceAudience.Private
-public class BackupTableInfo  {
-
-  /*
-   *  Table name for backup
-   */
-  private TableName table;
-
-  /*
-   *  Snapshot name for offline/online snapshot
-   */
-
-  private String snapshotName = null;
-
-  public BackupTableInfo() {
-
-  }
-
-  public BackupTableInfo(TableName table, String targetRootDir, String backupId) {
-    this.table = table;
-  }
-
-  public String getSnapshotName() {
-    return snapshotName;
-  }
-
-  public void setSnapshotName(String snapshotName) {
-    this.snapshotName = snapshotName;
-  }
-
-  public TableName getTable() {
-    return table;
-  }
-
-  public static BackupTableInfo convert(BackupProtos.BackupTableInfo proto) {
-    BackupTableInfo bs = new BackupTableInfo();
-    bs.table = ProtobufUtil.toTableName(proto.getTableName());
-    if (proto.hasSnapshotName()) {
-      bs.snapshotName = proto.getSnapshotName();
-    }
-    return bs;
-  }
-
-  public BackupProtos.BackupTableInfo toProto() {
-    BackupProtos.BackupTableInfo.Builder builder = BackupProtos.BackupTableInfo.newBuilder();
-    if (snapshotName != null) {
-      builder.setSnapshotName(snapshotName);
-    }
-    builder.setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toProtoTableName(table));
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
deleted file mode 100644
index 1c43e88..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * 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
-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/backup/backup_1396650096738/default/t1_dn/", where
-   * "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
-   * @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;
-  }
-
-  public static String getTableBackupDataDir(String backupRootDir, String backupId,
-      TableName tableName) {
-    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
-  }
-
-  public static Path getBackupPath(String backupRootDir, String backupId) {
-    return new Path(backupRootDir + Path.SEPARATOR + backupId);
-  }
-
-  /**
-   * 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/backup_1396650096738/default/t1_dn/", where
-   * "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
-   * @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));
-  }
-
-  /**
-   * 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/backup/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));
-  }
-
-  // TODO we do not keep WAL files anymore
-  // Move manifest file to other place
-  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
-      throws IOException {
-    Path manifestPath = null;
-
-    FileSystem fs = backupRootPath.getFileSystem(conf);
-    manifestPath =
-        new Path(getBackupPath(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 + ". File " + manifestPath + " does not exists. Did " + backupId
-              + " correspond to previously taken backup ?";
-      throw new IOException(errorMsg);
-    }
-    return manifestPath;
-  }
-
-  public static BackupManifest
-      getManifest(Configuration conf, Path backupRootPath, String backupId) throws IOException {
-    BackupManifest manifest =
-        new BackupManifest(conf, getManifestPath(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<TableName, BackupManifest> backupManifestMap,
-      TableName[] tableArray, Configuration conf, Path backupRootPath, String backupId)
-      throws IOException {
-    for (TableName tableName : tableArray) {
-      BackupManifest manifest = getManifest(conf, backupRootPath, backupId);
-      backupManifestMap.put(tableName, manifest);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
deleted file mode 100644
index 1becb75..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
-  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * Utility class for disabling Zk and client logging
- *
- */
-@InterfaceAudience.Private
-final class LogUtils {
-
-  private LogUtils() {
-  }
-
-  /**
-   * Disables Zk- and HBase client logging
-   * @param log
-   */
-  static void disableZkAndClientLoggers(Log log) {
-    // disable zookeeper log to avoid it mess up command output
-    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
-    zkLogger.setLevel(Level.OFF);
-    // disable hbase zookeeper tool log to avoid it mess up command output
-    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
-    hbaseZkLogger.setLevel(Level.OFF);
-    // disable hbase client log to avoid it mess up command output
-    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
-    hbaseClientLogger.setLevel(Level.OFF);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
deleted file mode 100644
index 82a1b56..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC;
-
-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.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupManager;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-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;
-
-/**
- *
- * Command-line entry point for restore operation
- *
- */
-@InterfaceAudience.Private
-public class RestoreDriver extends AbstractHBaseTool {
-
-  private static final Log LOG = LogFactory.getLog(RestoreDriver.class);
-  private CommandLine cmd;
-
-  private static final String USAGE_STRING =
-      "Usage: hbase restore <backup_path> <backup_id> [options]\n"
-          + "  backup_path     Path to a backup destination root\n"
-          + "  backup_id       Backup image ID to restore\n"
-          + "  table(s)        Comma-separated list of tables to restore\n";
-
-  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 {
-    // Check if backup is enabled
-    if (!BackupManager.isBackupEnabled(getConf())) {
-      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
-      return -1;
-    }
-
-    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
-
-    // 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");
-    }
-
-    if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) {
-      System.err.println("Options -s and -t are mutaully exclusive,"+
-          " you can not specify both of them.");
-      printToolUsage();
-      return -1;
-    }
-
-    if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) {
-      System.err.println("You have to specify either set name or table list to restore");
-      printToolUsage();
-      return -1;
-    }
-    // parse main restore command options
-    String[] remainArgs = cmd.getArgs();
-    if (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 BackupAdminImpl(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 = cmd.getOptionValue(OPTION_TABLE);
-      }
-
-      TableName[] sTableArray = BackupUtils.parseTableNames(tables);
-      TableName[] tTableArray = BackupUtils.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(BackupUtils.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<TableName> 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, OPTION_TABLE_LIST_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 (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;
-  }
-
-  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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
deleted file mode 100644
index 86fb963..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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;
-
-/**
- * Restore operation job interface Concrete implementation is provided by backup provider, see
- * {@link BackupRestoreFactory}
- */
-
-@InterfaceAudience.Private
-public interface RestoreJob 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;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
deleted file mode 100644
index de3ad5a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * POJO class for restore request
- */
-@InterfaceAudience.Private
-public class RestoreRequest {
-
-  public static class Builder {
-    RestoreRequest request;
-
-    public Builder() {
-      request = new RestoreRequest();
-    }
-
-    public Builder withBackupRootDir(String backupRootDir) {
-      request.setBackupRootDir(backupRootDir);
-      return this;
-    }
-
-    public Builder withBackupId(String backupId) {
-      request.setBackupId(backupId);
-      return this;
-    }
-
-    public Builder withCheck(boolean check) {
-      request.setCheck(check);
-      return this;
-    }
-
-    public Builder withFromTables(TableName[] fromTables) {
-      request.setFromTables(fromTables);
-      return this;
-    }
-
-    public Builder withToTables(TableName[] toTables) {
-      request.setToTables(toTables);
-      return this;
-    }
-
-    public Builder withOvewrite(boolean overwrite) {
-      request.setOverwrite(overwrite);
-      return this;
-    }
-
-
-    public RestoreRequest build() {
-      return request;
-    }
-  }
-
-  private String backupRootDir;
-  private String backupId;
-  private boolean check = false;
-  private TableName[] fromTables;
-  private TableName[] toTables;
-  private boolean overwrite = false;
-
-  private RestoreRequest() {
-  }
-
-  public String getBackupRootDir() {
-    return backupRootDir;
-  }
-
-  private RestoreRequest setBackupRootDir(String backupRootDir) {
-    this.backupRootDir = backupRootDir;
-    return this;
-  }
-
-  public String getBackupId() {
-    return backupId;
-  }
-
-  private RestoreRequest setBackupId(String backupId) {
-    this.backupId = backupId;
-    return this;
-  }
-
-  public boolean isCheck() {
-    return check;
-  }
-
-  private RestoreRequest setCheck(boolean check) {
-    this.check = check;
-    return this;
-  }
-
-  public TableName[] getFromTables() {
-    return fromTables;
-  }
-
-  private RestoreRequest setFromTables(TableName[] fromTables) {
-    this.fromTables = fromTables;
-    return this;
-  }
-
-  public TableName[] getToTables() {
-    return toTables;
-  }
-
-  private RestoreRequest setToTables(TableName[] toTables) {
-    this.toTables = toTables;
-    return this;
-  }
-
-  public boolean isOverwrite() {
-    return overwrite;
-  }
-
-  private RestoreRequest setOverwrite(boolean overwrite) {
-    this.overwrite = overwrite;
-    return this;
-  }
-}