Repository: hbase Updated Branches: refs/heads/HBASE-7912 c61f7c342 -> 182d66f8e
HBASE-14123 HBase Backup/Restore Phase 2 - drop deleted files Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/182d66f8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/182d66f8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/182d66f8 Branch: refs/heads/HBASE-7912 Commit: 182d66f8e8f7795ff4b7532690a7828f228dee64 Parents: c61f7c3 Author: tedyu <yuzhih...@gmail.com> Authored: Fri Apr 22 09:30:21 2016 -0700 Committer: tedyu <yuzhih...@gmail.com> Committed: Fri Apr 22 09:30:21 2016 -0700 ---------------------------------------------------------------------- .../hadoop/hbase/backup/impl/BackupContext.java | 402 ---------- .../hadoop/hbase/backup/impl/BackupStatus.java | 105 --- .../backup/impl/FullTableBackupProcedure.java | 745 ------------------- .../impl/IncrementalTableBackupProcedure.java | 325 -------- 4 files changed, 1577 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/182d66f8/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java deleted file mode 100644 index 06e66dc..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java +++ /dev/null @@ -1,402 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.backup.impl; - -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -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.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupContext.Builder; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus; - -/** - * An object to encapsulate the information for each backup request - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public class BackupContext { - - public Map<TableName, BackupStatus> getBackupStatusMap() { - return backupStatusMap; - } - - public void setBackupStatusMap(Map<TableName, BackupStatus> backupStatusMap) { - this.backupStatusMap = backupStatusMap; - } - - public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() { - return tableSetTimestampMap; - } - - public void setTableSetTimestampMap( - HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap) { - this.tableSetTimestampMap = tableSetTimestampMap; - } - - public String getHlogTargetDir() { - return hlogTargetDir; - } - - public void setType(BackupType type) { - this.type = type; - } - - public void setTargetRootDir(String targetRootDir) { - this.targetRootDir = targetRootDir; - } - - public void setTotalBytesCopied(long totalBytesCopied) { - this.totalBytesCopied = totalBytesCopied; - } - - // backup status flag - public static enum BackupState { - RUNNING, COMPLETE, FAILED, CANCELLED; - } - - public void setCancelled(boolean cancelled) { - this.state = BackupState.CANCELLED;; - } - - // backup phase - // for overall backup (for table list, some table may go online, while some may go offline) - protected static enum BackupPhase { - SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST; - } - - // backup id: a timestamp when we request the backup - private String backupId; - - // backup type, full or incremental - private BackupType type; - - // target root directory for storing the backup files - private String targetRootDir; - - // overall backup state - private BackupState state; - - // overall backup phase - private BackupPhase phase; - - // overall backup failure message - private String failedMsg; - - // backup status map for all tables - private Map<TableName, BackupStatus> backupStatusMap; - - // actual start timestamp of the backup process - private long startTs; - - // actual end timestamp of the backup process, could be fail or complete - private long endTs; - - // the total bytes of incremental logs copied - private long totalBytesCopied; - - // for incremental backup, the location of the backed-up hlogs - private String hlogTargetDir = null; - - // incremental backup file list - transient 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 - transient private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap; - - // backup progress in %% (0-100) - - private int progress; - - public BackupContext() { - } - - public BackupContext(String backupId, BackupType type, TableName[] tables, String targetRootDir) { - backupStatusMap = new HashMap<TableName, BackupStatus>(); - - this.backupId = backupId; - this.type = type; - this.targetRootDir = targetRootDir; - - this.addTables(tables); - - if (type == BackupType.INCREMENTAL) { - setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, backupId)); - } - - this.startTs = 0; - this.endTs = 0; - } - - /** - * Set progress string - * @param msg progress message - */ - - public void setProgress(int p) { - this.progress = p; - } - - /** - * Get current progress - */ - public int getProgress() { - return progress; - } - - - /** - * Has been marked as cancelled or not. - * @return True if marked as cancelled - */ - public boolean isCancelled() { - return this.state == BackupState.CANCELLED; - } - - public String getBackupId() { - return backupId; - } - - public void setBackupId(String backupId) { - this.backupId = backupId; - } - - public BackupStatus getBackupStatus(TableName table) { - return this.backupStatusMap.get(table); - } - - public String getFailedMsg() { - return failedMsg; - } - - public void setFailedMsg(String failedMsg) { - this.failedMsg = failedMsg; - } - - public long getStartTs() { - return startTs; - } - - public void setStartTs(long startTs) { - this.startTs = startTs; - } - - public long getEndTs() { - return endTs; - } - - public void setEndTs(long endTs) { - this.endTs = endTs; - } - - public long getTotalBytesCopied() { - return totalBytesCopied; - } - - public BackupState getState() { - return state; - } - - public void setState(BackupState flag) { - this.state = flag; - } - - public BackupPhase getPhase() { - return phase; - } - - public void setPhase(BackupPhase phase) { - this.phase = phase; - } - - public BackupType getType() { - return type; - } - - public void setSnapshotName(TableName table, String snapshotName) { - this.backupStatusMap.get(table).setSnapshotName(snapshotName); - } - - public String getSnapshotName(TableName table) { - return this.backupStatusMap.get(table).getSnapshotName(); - } - - public List<String> getSnapshotNames() { - List<String> snapshotNames = new ArrayList<String>(); - for (BackupStatus backupStatus : this.backupStatusMap.values()) { - snapshotNames.add(backupStatus.getSnapshotName()); - } - return snapshotNames; - } - - public Set<TableName> getTables() { - return this.backupStatusMap.keySet(); - } - - public List<TableName> getTableNames() { - return new ArrayList<TableName>(backupStatusMap.keySet()); - } - - public void addTables(TableName[] tables) { - for (TableName table : tables) { - BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId); - this.backupStatusMap.put(table, backupStatus); - } - } - - public String getTargetRootDir() { - return targetRootDir; - } - - public void setHlogTargetDir(String hlogTagetDir) { - this.hlogTargetDir = hlogTagetDir; - } - - public String getHLogTargetDir() { - return hlogTargetDir; - } - - public List<String> getIncrBackupFileList() { - return incrBackupFileList; - } - - public List<String> setIncrBackupFileList(List<String> incrBackupFileList) { - this.incrBackupFileList = incrBackupFileList; - return this.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, BackupStatus> entry : this.backupStatusMap.entrySet()) { - if (snapshotName.equals(entry.getValue().getSnapshotName())) { - return entry.getKey(); - } - } - return null; - } - - BackupProtos.BackupContext toBackupContext() { - BackupProtos.BackupContext.Builder builder = - BackupProtos.BackupContext.newBuilder(); - builder.setBackupId(getBackupId()); - setBackupStatusMap(builder); - builder.setEndTs(getEndTs()); - if(getFailedMsg() != null){ - builder.setFailedMessage(getFailedMsg()); - } - if(getState() != null){ - builder.setState(BackupProtos.BackupContext.BackupState.valueOf(getState().name())); - } - if(getPhase() != null){ - builder.setPhase(BackupProtos.BackupContext.BackupPhase.valueOf(getPhase().name())); - } - if(getHLogTargetDir() != null){ - builder.setHlogTargetDir(getHLogTargetDir()); - } - - builder.setProgress(getProgress()); - builder.setStartTs(getStartTs()); - builder.setTargetRootDir(getTargetRootDir()); - builder.setTotalBytesCopied(getTotalBytesCopied()); - builder.setType(BackupProtos.BackupType.valueOf(getType().name())); - return builder.build(); - } - - public byte[] toByteArray() throws IOException { - return toBackupContext().toByteArray(); - } - - private void setBackupStatusMap(Builder builder) { - for (Entry<TableName, BackupStatus> entry: backupStatusMap.entrySet()) { - builder.addTableBackupStatus(entry.getValue().toProto()); - } - } - - public static BackupContext fromByteArray(byte[] data) throws IOException { - return fromProto(BackupProtos.BackupContext.parseFrom(data)); - } - - public static BackupContext fromStream(final InputStream stream) throws IOException { - return fromProto(BackupProtos.BackupContext.parseDelimitedFrom(stream)); - } - - static BackupContext fromProto(BackupProtos.BackupContext proto) { - BackupContext context = new BackupContext(); - context.setBackupId(proto.getBackupId()); - context.setBackupStatusMap(toMap(proto.getTableBackupStatusList())); - context.setEndTs(proto.getEndTs()); - if(proto.hasFailedMessage()) { - context.setFailedMsg(proto.getFailedMessage()); - } - if(proto.hasState()) { - context.setState(BackupContext.BackupState.valueOf(proto.getState().name())); - } - if(proto.hasHlogTargetDir()) { - context.setHlogTargetDir(proto.getHlogTargetDir()); - } - if(proto.hasPhase()) { - context.setPhase(BackupPhase.valueOf(proto.getPhase().name())); - } - if(proto.hasProgress()) { - context.setProgress(proto.getProgress()); - } - context.setStartTs(proto.getStartTs()); - context.setTargetRootDir(proto.getTargetRootDir()); - context.setTotalBytesCopied(proto.getTotalBytesCopied()); - context.setType(BackupType.valueOf(proto.getType().name())); - return context; - } - - private static Map<TableName, BackupStatus> toMap(List<TableBackupStatus> list) { - HashMap<TableName, BackupStatus> map = new HashMap<>(); - for (TableBackupStatus tbs : list){ - map.put(ProtobufUtil.toTableName(tbs.getTable()), BackupStatus.convert(tbs)); - } - return map; - } - -} http://git-wip-us.apache.org/repos/asf/hbase/blob/182d66f8/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 deleted file mode 100644 index 6e54994..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.backup.impl; - -import java.io.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(); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/182d66f8/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java deleted file mode 100644 index 175f2bb..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupProcedure.java +++ /dev/null @@ -1,745 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.backup.impl; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicBoolean; - -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.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.BackupClientUtil; -import org.apache.hadoop.hbase.backup.BackupRestoreFactory; -import org.apache.hadoop.hbase.backup.BackupType; -import org.apache.hadoop.hbase.backup.HBackupFileSystem; -import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupPhase; -import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupState; -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.master.procedure.MasterProcedureEnv; -import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; -import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface; -import org.apache.hadoop.hbase.procedure.MasterProcedureManager; -import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.FullTableBackupState; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; - -@InterfaceAudience.Private -public class FullTableBackupProcedure - extends StateMachineProcedure<MasterProcedureEnv, FullTableBackupState> - implements TableProcedureInterface { - private static final Log LOG = LogFactory.getLog(FullTableBackupProcedure.class); - - private final AtomicBoolean aborted = new AtomicBoolean(false); - private Configuration conf; - private String backupId; - private List<TableName> tableList; - private String targetRootDir; - HashMap<String, Long> newTimestamps = null; - - private BackupManager backupManager; - private BackupContext backupContext; - - public FullTableBackupProcedure() { - // Required by the Procedure framework to create the procedure on replay - } - - public FullTableBackupProcedure(final MasterProcedureEnv env, - final String backupId, List<TableName> tableList, String targetRootDir, final int workers, - final long bandwidth) throws IOException { - backupManager = new BackupManager(env.getMasterConfiguration()); - this.backupId = backupId; - this.tableList = tableList; - this.targetRootDir = targetRootDir; - backupContext = - backupManager.createBackupContext(backupId, BackupType.FULL, tableList, targetRootDir); - if (tableList == null || tableList.isEmpty()) { - this.tableList = new ArrayList<>(backupContext.getTables()); - } - } - - @Override - public byte[] getResult() { - return backupId.getBytes(); - } - - /** - * Begin the overall backup. - * @param backupContext backup context - * @throws IOException exception - */ - static void beginBackup(BackupManager backupManager, BackupContext backupContext) - throws IOException { - backupManager.setBackupContext(backupContext); - // set the start timestamp of the overall backup - long startTs = EnvironmentEdgeManager.currentTime(); - backupContext.setStartTs(startTs); - // set overall backup status: ongoing - backupContext.setState(BackupState.RUNNING); - LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + "."); - - backupManager.updateBackupStatus(backupContext); - if (LOG.isDebugEnabled()) { - LOG.debug("Backup session " + backupContext.getBackupId() + " has been started."); - } - } - - private static String getMessage(Exception e) { - String msg = e.getMessage(); - if (msg == null || msg.equals("")) { - msg = e.getClass().getName(); - } - return msg; - } - - /** - * Delete HBase snapshot for backup. - * @param backupCtx backup context - * @throws Exception exception - */ - private static void deleteSnapshot(final MasterProcedureEnv env, - BackupContext backupCtx, Configuration conf) - throws IOException { - LOG.debug("Trying to delete snapshot for full backup."); - for (String snapshotName : backupCtx.getSnapshotNames()) { - if (snapshotName == null) { - continue; - } - LOG.debug("Trying to delete snapshot: " + snapshotName); - HBaseProtos.SnapshotDescription.Builder builder = - HBaseProtos.SnapshotDescription.newBuilder(); - builder.setName(snapshotName); - try { - env.getMasterServices().getSnapshotManager().deleteSnapshot(builder.build()); - } catch (IOException ioe) { - LOG.debug("when deleting snapshot " + snapshotName, ioe); - } - LOG.debug("Deleting the snapshot " + snapshotName + " for backup " - + backupCtx.getBackupId() + " succeeded."); - } - } - - /** - * Clean up directories with prefix "exportSnapshot-", which are generated when exporting - * snapshots. - * @throws IOException exception - */ - private static void cleanupExportSnapshotLog(Configuration conf) throws IOException { - FileSystem fs = FSUtils.getCurrentFileSystem(conf); - Path stagingDir = - new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory() - .toString())); - FileStatus[] files = FSUtils.listStatus(fs, stagingDir); - if (files == null) { - return; - } - for (FileStatus file : files) { - if (file.getPath().getName().startsWith("exportSnapshot-")) { - LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); - if (FSUtils.delete(fs, file.getPath(), true) == false) { - LOG.warn("Can not delete " + file.getPath()); - } - } - } - } - - /** - * Clean up the uncompleted data at target directory if the ongoing backup has already entered the - * copy phase. - */ - static void cleanupTargetDir(BackupContext backupContext, Configuration conf) { - try { - // clean up the uncompleted data at target directory if the ongoing backup has already entered - // the copy phase - LOG.debug("Trying to cleanup up target dir. Current backup phase: " - + backupContext.getPhase()); - if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY) - || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY) - || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) { - FileSystem outputFs = - FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf); - - // now treat one backup as a transaction, clean up data that has been partially copied at - // table level - for (TableName table : backupContext.getTables()) { - Path targetDirPath = - new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(), - backupContext.getBackupId(), table)); - if (outputFs.delete(targetDirPath, true)) { - LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString() - + " done."); - } else { - LOG.info("No data has been copied to " + targetDirPath.toString() + "."); - } - - Path tableDir = targetDirPath.getParent(); - FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir); - if (backups == null || backups.length == 0) { - outputFs.delete(tableDir, true); - LOG.debug(tableDir.toString() + " is empty, remove it."); - } - } - } - - } catch (IOException e1) { - LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at " - + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); - } - } - - /** - * Fail the overall backup. - * @param backupContext backup context - * @param e exception - * @throws Exception exception - */ - static void failBackup(final MasterProcedureEnv env, BackupContext backupContext, - BackupManager backupManager, Exception e, - String msg, BackupType type, Configuration conf) throws IOException { - LOG.error(msg + getMessage(e)); - // If this is a cancel exception, then we've already cleaned. - - if (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 are snapshots taken - // and also clean up export snapshot log files if exist - if (type == BackupType.FULL) { - deleteSnapshot(env, backupContext, conf); - cleanupExportSnapshotLog(conf); - } - - // clean up the uncompleted data at target directory if the ongoing backup has already entered - // the copy phase - // For incremental backup, DistCp logs will be cleaned with the targetDir. - cleanupTargetDir(backupContext, conf); - - LOG.info("Backup " + backupContext.getBackupId() + " failed."); - } - - /** - * Do snapshot copy. - * @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. - // 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."); - } - } - - /** - * Add manifest for the current backup. The manifest is stored - * within the table backup directory. - * @param backupContext The current backup context - * @throws IOException exception - * @throws BackupException exception - */ - private static void addManifest(BackupContext backupContext, BackupManager backupManager, - BackupType type, Configuration conf) 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 = backupManager.getAncestors(backupContext, table); - for (BackupImage image : ancestors) { - manifest.addDependentImage(image); - } - - if (type == BackupType.INCREMENTAL) { - // We'll store the log timestamps for this table only in its manifest. - HashMap<TableName, HashMap<String, Long>> tableTimestampMap = - new HashMap<TableName, HashMap<String, Long>>(); - tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table)); - manifest.setIncrTimestampMap(tableTimestampMap); - ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupContext); - for (BackupImage image : ancestorss) { - manifest.addDependentImage(image); - } - } - // TODO - // manifest.setRelativeWALReferences(backupContext.getRelWALRefs()); - 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 (type == 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 = backupManager.getAncestors(backupContext); - for (BackupImage image : ancestors) { - manifest.addDependentImage(image); - } - // TODO - // manifest.setRelativeWALReferences(backupContext.getRelWALRefs()); - manifest.store(conf); - } - } - - /** - * Get backup request meta data dir as string. - * @param backupContext backup context - * @return meta data dir - */ - private static 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(); - } - - /** - * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying - * hlogs. - * @throws IOException exception - */ - private static void cleanupDistCpLog(BackupContext backupContext, Configuration conf) - throws IOException { - Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent(); - FileSystem fs = FileSystem.get(rootPath.toUri(), conf); - FileStatus[] files = FSUtils.listStatus(fs, rootPath); - if (files == null) { - return; - } - for (FileStatus file : files) { - if (file.getPath().getName().startsWith("_distcp_logs")) { - LOG.debug("Delete log files of DistCp: " + file.getPath().getName()); - FSUtils.delete(fs, file.getPath(), true); - } - } - } - - /** - * Complete the overall backup. - * @param backupContext backup context - * @throws Exception exception - */ - static void completeBackup(final MasterProcedureEnv env, BackupContext backupContext, - BackupManager backupManager, BackupType type, Configuration conf) throws IOException { - // set the complete timestamp of the overall backup - backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); - // set overall backup status: complete - backupContext.setState(BackupState.COMPLETE); - // add and store the manifest for the backup - addManifest(backupContext, backupManager, type, conf); - - // after major steps done and manifest persisted, do convert if needed for incremental backup - /* in-fly convert code here, provided by future jira */ - LOG.debug("in-fly convert code here, provided by future jira"); - - // compose the backup complete data - String backupCompleteData = - obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs() - + ",completets=" + backupContext.getEndTs() + ",bytescopied=" - + backupContext.getTotalBytesCopied(); - if (LOG.isDebugEnabled()) { - LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData); - } - backupManager.updateBackupStatus(backupContext); - - // when full backup is done: - // - delete HBase snapshot - // - clean up directories with prefix "exportSnapshot-", which are generated when exporting - // snapshots - if (type == BackupType.FULL) { - deleteSnapshot(env, backupContext, conf); - cleanupExportSnapshotLog(conf); - } else if (type == BackupType.INCREMENTAL) { - cleanupDistCpLog(backupContext, conf); - } - - LOG.info("Backup " + backupContext.getBackupId() + " completed."); - } - - /** - * Wrap a SnapshotDescription for a target table. - * @param table table - * @return a SnapshotDescription especially for backup. - */ - static SnapshotDescription wrapSnapshotDescription(TableName tableName, String snapshotName) { - // Mock a SnapshotDescription from backupContext to call SnapshotManager function, - // Name it in the format "snapshot_<timestamp>_<table>" - HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder(); - builder.setTable(tableName.getNameAsString()); - builder.setName(snapshotName); - HBaseProtos.SnapshotDescription backupSnapshot = builder.build(); - - LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName() - + " from backupContext to request snapshot for backup."); - - return backupSnapshot; - } - - @Override - protected Flow executeFromState(final MasterProcedureEnv env, final FullTableBackupState state) - throws InterruptedException { - if (conf == null) { - conf = env.getMasterConfiguration(); - } - if (backupManager == null) { - try { - backupManager = new BackupManager(env.getMasterConfiguration()); - } catch (IOException ioe) { - setFailure("full backup", ioe); - return Flow.NO_MORE_STATE; - } - } - if (LOG.isTraceEnabled()) { - LOG.trace(this + " execute state=" + state); - } - try { - switch (state) { - case PRE_SNAPSHOT_TABLE: - beginBackup(backupManager, backupContext); - String savedStartCode = null; - boolean firstBackup = false; - // do snapshot for full table backup - - try { - savedStartCode = backupManager.readBackupStartCode(); - firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; - 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 ..."); - MasterProcedureManager mpm = env.getMasterServices().getMasterProcedureManagerHost() - .getProcedureManager(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE); - Map<String, String> props= new HashMap<String, String>(); - long waitTime = MasterProcedureUtil.execProcedure(mpm, - LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, - LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); - MasterProcedureUtil.waitForProcedure(mpm, - LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, - LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props, waitTime, - conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, - HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER), - conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE)); - - 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); - } - } catch (BackupException e) { - // fail the overall backup and return - failBackup(env, backupContext, backupManager, e, "Unexpected BackupException : ", - BackupType.FULL, conf); - return Flow.NO_MORE_STATE; - } - setNextState(FullTableBackupState.SNAPSHOT_TABLES); - break; - case SNAPSHOT_TABLES: - for (TableName tableName : tableList) { - String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) - + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString(); - HBaseProtos.SnapshotDescription backupSnapshot; - - // wrap a SnapshotDescription for offline/online snapshot - backupSnapshot = wrapSnapshotDescription(tableName,snapshotName); - try { - env.getMasterServices().getSnapshotManager().deleteSnapshot(backupSnapshot); - } catch (IOException e) { - LOG.debug("Unable to delete " + snapshotName, e); - } - // Kick off snapshot for backup - try { - env.getMasterServices().getSnapshotManager().takeSnapshot(backupSnapshot); - } catch (IOException e) { - LOG.debug("Unable to take snapshot: " + snapshotName, e); - } - long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout( - env.getMasterConfiguration(), - backupSnapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); - BackupUtil.waitForSnapshot(backupSnapshot, waitTime, - env.getMasterServices().getSnapshotManager(), env.getMasterConfiguration()); - // set the snapshot name in BackupStatus of this table, only after snapshot success. - backupContext.setSnapshotName(tableName, backupSnapshot.getName()); - } - setNextState(FullTableBackupState.SNAPSHOT_COPY); - break; - case SNAPSHOT_COPY: - // do snapshot copy - LOG.debug("snapshot copy for " + backupId); - try { - this.snapshotCopy(backupContext); - } catch (Exception e) { - // fail the overall backup and return - failBackup(env, backupContext, backupManager, e, "Unexpected BackupException : ", - BackupType.FULL, conf); - return Flow.NO_MORE_STATE; - } - // Updates incremental backup table set - backupManager.addIncrementalBackupTableSet(backupContext.getTables()); - setNextState(FullTableBackupState.BACKUP_COMPLETE); - break; - - case BACKUP_COMPLETE: - // 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); - // 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 = - BackupClientUtil.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap)); - backupManager.writeBackupStartCode(newStartCode); - - // backup complete - completeBackup(env, backupContext, backupManager, BackupType.FULL, conf); - return Flow.NO_MORE_STATE; - - default: - throw new UnsupportedOperationException("unhandled state=" + state); - } - } catch (IOException e) { - LOG.error("Backup failed in " + state); - setFailure("snapshot-table", e); - } - return Flow.HAS_MORE_STATE; - } - - @Override - protected void rollbackState(final MasterProcedureEnv env, final FullTableBackupState state) - throws IOException { - if (state != FullTableBackupState.PRE_SNAPSHOT_TABLE) { - deleteSnapshot(env, backupContext, conf); - cleanupExportSnapshotLog(conf); - } - - // clean up the uncompleted data at target directory if the ongoing backup has already entered - // the copy phase - // For incremental backup, DistCp logs will be cleaned with the targetDir. - if (state == FullTableBackupState.SNAPSHOT_COPY) { - cleanupTargetDir(backupContext, conf); - } - } - - @Override - protected FullTableBackupState getState(final int stateId) { - return FullTableBackupState.valueOf(stateId); - } - - @Override - protected int getStateId(final FullTableBackupState state) { - return state.getNumber(); - } - - @Override - protected FullTableBackupState getInitialState() { - return FullTableBackupState.PRE_SNAPSHOT_TABLE; - } - - @Override - protected void setNextState(final FullTableBackupState state) { - if (aborted.get()) { - setAbortFailure("backup-table", "abort requested"); - } else { - super.setNextState(state); - } - } - - @Override - public boolean abort(final MasterProcedureEnv env) { - aborted.set(true); - return true; - } - - @Override - public void toStringClassDetails(StringBuilder sb) { - sb.append(getClass().getSimpleName()); - sb.append(" (targetRootDir="); - sb.append(targetRootDir); - sb.append(")"); - } - - BackupProtos.BackupProcContext toBackupContext() { - BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder(); - ctxBuilder.setCtx(backupContext.toBackupContext()); - if (newTimestamps != null && !newTimestamps.isEmpty()) { - BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder(); - for (Entry<String, Long> entry : newTimestamps.entrySet()) { - tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue()); - ctxBuilder.addServerTimestamp(tsBuilder.build()); - } - } - return ctxBuilder.build(); - } - - @Override - public void serializeStateData(final OutputStream stream) throws IOException { - super.serializeStateData(stream); - - BackupProtos.BackupProcContext backupProcCtx = toBackupContext(); - backupProcCtx.writeDelimitedTo(stream); - } - - @Override - public void deserializeStateData(final InputStream stream) throws IOException { - super.deserializeStateData(stream); - - BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream); - backupContext = BackupContext.fromProto(proto.getCtx()); - backupId = backupContext.getBackupId(); - targetRootDir = backupContext.getTargetRootDir(); - tableList = backupContext.getTableNames(); - List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList(); - if (svrTimestamps != null && !svrTimestamps.isEmpty()) { - newTimestamps = new HashMap<>(); - for (ServerTimestamp ts : svrTimestamps) { - newTimestamps.put(ts.getServer(), ts.getTimestamp()); - } - } - } - - @Override - public TableName getTableName() { - return TableName.BACKUP_TABLE_NAME; - } - - @Override - public TableOperationType getTableOperationType() { - return TableOperationType.BACKUP; - } - - @Override - protected boolean acquireLock(final MasterProcedureEnv env) { - if (env.waitInitialized(this)) { - return false; - } - return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME); - } - - @Override - protected void releaseLock(final MasterProcedureEnv env) { - env.getProcedureQueue().releaseTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/182d66f8/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java deleted file mode 100644 index 8c12582..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupProcedure.java +++ /dev/null @@ -1,325 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.backup.impl; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.BackupClientUtil; -import org.apache.hadoop.hbase.backup.BackupRestoreFactory; -import org.apache.hadoop.hbase.backup.BackupType; -import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupPhase; -import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupState; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; -import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface; -import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.IncrementalTableBackupState; -import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp; - -@InterfaceAudience.Private -public class IncrementalTableBackupProcedure - extends StateMachineProcedure<MasterProcedureEnv, IncrementalTableBackupState> - implements TableProcedureInterface { - private static final Log LOG = LogFactory.getLog(IncrementalTableBackupProcedure.class); - - private final AtomicBoolean aborted = new AtomicBoolean(false); - private Configuration conf; - private String backupId; - private List<TableName> tableList; - private String targetRootDir; - HashMap<String, Long> newTimestamps = null; - - private BackupManager backupManager; - private BackupContext backupContext; - - public IncrementalTableBackupProcedure() { - // Required by the Procedure framework to create the procedure on replay - } - - public IncrementalTableBackupProcedure(final MasterProcedureEnv env, - final String backupId, - List<TableName> tableList, String targetRootDir, final int workers, - final long bandwidth) throws IOException { - backupManager = new BackupManager(env.getMasterConfiguration()); - this.backupId = backupId; - this.tableList = tableList; - this.targetRootDir = targetRootDir; - backupContext = backupManager.createBackupContext(backupId, BackupType.INCREMENTAL, tableList, - targetRootDir); - } - - @Override - public byte[] getResult() { - return backupId.getBytes(); - } - - 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; - } - - /** - * 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."); - } - - @Override - protected Flow executeFromState(final MasterProcedureEnv env, - final IncrementalTableBackupState state) - throws InterruptedException { - if (conf == null) { - conf = env.getMasterConfiguration(); - } - if (backupManager == null) { - try { - backupManager = new BackupManager(env.getMasterConfiguration()); - } catch (IOException ioe) { - setFailure("incremental backup", ioe); - } - } - if (LOG.isTraceEnabled()) { - LOG.trace(this + " execute state=" + state); - } - try { - switch (state) { - case PREPARE_INCREMENTAL: - FullTableBackupProcedure.beginBackup(backupManager, backupContext); - LOG.debug("For incremental backup, current table set is " - + backupManager.getIncrementalBackupTableSet()); - try { - IncrementalBackupManager incrBackupManager =new IncrementalBackupManager(backupManager); - - newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext); - } catch (Exception e) { - // fail the overall backup and return - FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e, - "Unexpected Exception : ", BackupType.INCREMENTAL, conf); - } - - setNextState(IncrementalTableBackupState.INCREMENTAL_COPY); - break; - case INCREMENTAL_COPY: - try { - // copy out the table and region info files for each table - BackupUtil.copyTableRegionInfo(backupContext, conf); - incrementalCopy(backupContext); - // Save list of WAL files copied - backupManager.recordWALFiles(backupContext.getIncrBackupFileList()); - } catch (Exception e) { - // fail the overall backup and return - FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e, - "Unexpected exception doing incremental copy : ", BackupType.INCREMENTAL, conf); - } - setNextState(IncrementalTableBackupState.INCR_BACKUP_COMPLETE); - break; - case INCR_BACKUP_COMPLETE: - // 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); - // 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 = BackupClientUtil - .getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap)); - backupManager.writeBackupStartCode(newStartCode); - // backup complete - FullTableBackupProcedure.completeBackup(env, backupContext, backupManager, - BackupType.INCREMENTAL, conf); - return Flow.NO_MORE_STATE; - - default: - throw new UnsupportedOperationException("unhandled state=" + state); - } - } catch (IOException e) { - setFailure("snapshot-table", e); - } - return Flow.HAS_MORE_STATE; - } - - @Override - protected void rollbackState(final MasterProcedureEnv env, - final IncrementalTableBackupState state) throws IOException { - // 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. - FullTableBackupProcedure.cleanupTargetDir(backupContext, conf); - } - - @Override - protected IncrementalTableBackupState getState(final int stateId) { - return IncrementalTableBackupState.valueOf(stateId); - } - - @Override - protected int getStateId(final IncrementalTableBackupState state) { - return state.getNumber(); - } - - @Override - protected IncrementalTableBackupState getInitialState() { - return IncrementalTableBackupState.PREPARE_INCREMENTAL; - } - - @Override - protected void setNextState(final IncrementalTableBackupState state) { - if (aborted.get()) { - setAbortFailure("snapshot-table", "abort requested"); - } else { - super.setNextState(state); - } - } - - @Override - public boolean abort(final MasterProcedureEnv env) { - aborted.set(true); - return true; - } - - @Override - public void toStringClassDetails(StringBuilder sb) { - sb.append(getClass().getSimpleName()); - sb.append(" (targetRootDir="); - sb.append(targetRootDir); - sb.append(")"); - } - - BackupProtos.BackupProcContext toBackupContext() { - BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder(); - ctxBuilder.setCtx(backupContext.toBackupContext()); - if (newTimestamps != null && !newTimestamps.isEmpty()) { - BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder(); - for (Entry<String, Long> entry : newTimestamps.entrySet()) { - tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue()); - ctxBuilder.addServerTimestamp(tsBuilder.build()); - } - } - return ctxBuilder.build(); - } - - @Override - public void serializeStateData(final OutputStream stream) throws IOException { - super.serializeStateData(stream); - - BackupProtos.BackupProcContext backupProcCtx = toBackupContext(); - backupProcCtx.writeDelimitedTo(stream); - } - - @Override - public void deserializeStateData(final InputStream stream) throws IOException { - super.deserializeStateData(stream); - - BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream); - backupContext = BackupContext.fromProto(proto.getCtx()); - backupId = backupContext.getBackupId(); - targetRootDir = backupContext.getTargetRootDir(); - tableList = backupContext.getTableNames(); - List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList(); - if (svrTimestamps != null && !svrTimestamps.isEmpty()) { - newTimestamps = new HashMap<>(); - for (ServerTimestamp ts : svrTimestamps) { - newTimestamps.put(ts.getServer(), ts.getTimestamp()); - } - } - } - - @Override - public TableName getTableName() { - return TableName.BACKUP_TABLE_NAME; - } - - @Override - public TableOperationType getTableOperationType() { - return TableOperationType.BACKUP; - } - - @Override - protected boolean acquireLock(final MasterProcedureEnv env) { - if (env.waitInitialized(this)) { - return false; - } - return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME); - } - - @Override - protected void releaseLock(final MasterProcedureEnv env) { - env.getProcedureQueue().releaseTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME); - } -}