From fd11b9bfcf2074ff0910ca1278f8a2fb66f0ad58 Mon Sep 17 00:00:00 2001 From: Mallikarjun Date: Tue, 24 Jan 2023 06:21:17 +0530 Subject: [PATCH] HBASE-27238 Backport backup restore to 2.x (#4770) Signed-off-by: Bryan Beaudreault --- bin/hbase | 28 + hbase-assembly/pom.xml | 4 + .../src/main/assembly/hadoop-three-compat.xml | 1 + .../src/main/assembly/hadoop-two-compat.xml | 1 + hbase-backup/pom.xml | 307 +++ .../hadoop/hbase/backup/BackupAdmin.java | 134 ++ .../hbase/backup/BackupClientFactory.java | 56 + .../hadoop/hbase/backup/BackupCopyJob.java | 52 + .../hadoop/hbase/backup/BackupDriver.java | 211 ++ .../hbase/backup/BackupHFileCleaner.java | 180 ++ .../hadoop/hbase/backup/BackupInfo.java | 578 +++++ .../hadoop/hbase/backup/BackupMergeJob.java | 37 + .../hadoop/hbase/backup/BackupObserver.java | 107 + .../hadoop/hbase/backup/BackupRequest.java | 151 ++ .../hbase/backup/BackupRestoreConstants.java | 140 ++ .../hbase/backup/BackupRestoreFactory.java | 78 + .../hadoop/hbase/backup/BackupTableInfo.java | 79 + .../hbase/backup/HBackupFileSystem.java | 158 ++ .../hadoop/hbase/backup/RestoreDriver.java | 270 +++ .../hadoop/hbase/backup/RestoreJob.java | 43 + .../hadoop/hbase/backup/RestoreRequest.java | 133 ++ .../hbase/backup/impl/BackupAdminImpl.java | 716 +++++++ .../hbase/backup/impl/BackupCommands.java | 1123 ++++++++++ .../hbase/backup/impl/BackupException.java | 82 + .../hbase/backup/impl/BackupManager.java | 521 +++++ .../hbase/backup/impl/BackupManifest.java | 664 ++++++ .../hbase/backup/impl/BackupSystemTable.java | 1877 +++++++++++++++++ .../impl/ExclusiveOperationException.java | 31 + .../backup/impl/FullTableBackupClient.java | 226 ++ .../backup/impl/IncrementalBackupManager.java | 283 +++ .../impl/IncrementalTableBackupClient.java | 436 ++++ .../backup/impl/RestoreTablesClient.java | 258 +++ .../hbase/backup/impl/TableBackupClient.java | 423 ++++ .../mapreduce/MapReduceBackupCopyJob.java | 428 ++++ .../mapreduce/MapReduceBackupMergeJob.java | 399 ++++ .../mapreduce/MapReduceHFileSplitterJob.java | 169 ++ .../backup/mapreduce/MapReduceRestoreJob.java | 119 ++ .../hbase/backup/master/BackupLogCleaner.java | 179 ++ .../master/LogRollMasterProcedureManager.java | 173 ++ .../LogRollBackupSubprocedure.java | 167 ++ .../LogRollBackupSubprocedurePool.java | 137 ++ .../LogRollRegionServerProcedureManager.java | 175 ++ .../hadoop/hbase/backup/util/BackupSet.java | 57 + .../hadoop/hbase/backup/util/BackupUtils.java | 742 +++++++ .../hadoop/hbase/backup/util/RestoreTool.java | 522 +++++ .../hadoop/hbase/backup/TestBackupBase.java | 509 +++++ .../hbase/backup/TestBackupBoundaryTests.java | 98 + .../backup/TestBackupCommandLineTool.java | 444 ++++ .../hadoop/hbase/backup/TestBackupDelete.java | 161 ++ .../hbase/backup/TestBackupDeleteRestore.java | 81 + .../backup/TestBackupDeleteWithFailures.java | 199 ++ .../hbase/backup/TestBackupDescribe.java | 110 + .../hbase/backup/TestBackupHFileCleaner.java | 148 ++ .../hbase/backup/TestBackupManager.java | 136 ++ .../hadoop/hbase/backup/TestBackupMerge.java | 127 ++ .../backup/TestBackupMultipleDeletes.java | 162 ++ .../hadoop/hbase/backup/TestBackupRepair.java | 92 + .../hbase/backup/TestBackupShowHistory.java | 141 ++ .../hbase/backup/TestBackupSmallTests.java | 60 + .../backup/TestBackupStatusProgress.java | 97 + .../hbase/backup/TestBackupSystemTable.java | 504 +++++ .../hadoop/hbase/backup/TestBackupUtils.java | 81 + .../hadoop/hbase/backup/TestFullBackup.java | 63 + .../hbase/backup/TestFullBackupSet.java | 100 + .../backup/TestFullBackupSetRestoreSet.java | 131 ++ .../backup/TestFullBackupWithFailures.java | 82 + .../hadoop/hbase/backup/TestFullRestore.java | 337 +++ .../hbase/backup/TestIncrementalBackup.java | 219 ++ .../TestIncrementalBackupDeleteTable.java | 129 ++ ...estIncrementalBackupMergeWithFailures.java | 350 +++ .../TestIncrementalBackupWithFailures.java | 165 ++ .../hadoop/hbase/backup/TestRemoteBackup.java | 153 ++ .../hbase/backup/TestRemoteRestore.java | 75 + .../backup/TestRepairAfterFailedDelete.java | 95 + .../backup/TestRestoreBoundaryTests.java | 82 + .../hbase/backup/TestSystemTableSnapshot.java | 58 + .../backup/master/TestBackupLogCleaner.java | 135 ++ .../src/test/resources/hbase-site.xml | 170 ++ hbase-it/pom.xml | 4 + .../hbase/IntegrationTestBackupRestore.java | 439 ++++ .../hbase/mapreduce/HFileOutputFormat2.java | 28 +- .../hadoop/hbase/mapreduce/WALPlayer.java | 100 +- .../TestCellBasedHFileOutputFormat2.java | 26 +- .../mapreduce/TestHFileOutputFormat2.java | 27 +- .../src/main/protobuf/Backup.proto | 121 ++ .../ProcedureCoordinationManager.java | 35 + .../ZKProcedureCoordinationManager.java | 42 + .../hadoop/hbase/util/FSTableDescriptors.java | 2 +- pom.xml | 6 + 89 files changed, 18930 insertions(+), 49 deletions(-) create mode 100644 hbase-backup/pom.xml create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/ExclusiveOperationException.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupManager.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSmallTests.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupUtils.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java create mode 100644 hbase-backup/src/test/resources/hbase-site.xml create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java create mode 100644 hbase-protocol-shaded/src/main/protobuf/Backup.proto create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinationManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinationManager.java diff --git a/bin/hbase b/bin/hbase index 53854f79417f..b1369ef576a6 100755 --- a/bin/hbase +++ b/bin/hbase @@ -100,6 +100,8 @@ show_usage() { echo " ltt Run LoadTestTool" echo " canary Run the Canary tool" echo " version Print the version" + echo " backup Backup tables for recovery" + echo " restore Restore tables from existing backup image" echo " completebulkload Run BulkLoadHFiles tool" echo " regionsplitter Run RegionSplitter tool" echo " rowcounter Run RowCounter tool" @@ -704,6 +706,32 @@ elif [ "$COMMAND" = "canary" ] ; then HBASE_OPTS="$HBASE_OPTS $HBASE_CANARY_OPTS" elif [ "$COMMAND" = "version" ] ; then CLASS='org.apache.hadoop.hbase.util.VersionInfo' +elif [ "$COMMAND" = "backup" ] ; then + CLASS='org.apache.hadoop.hbase.backup.BackupDriver' + if [ -n "${shaded_jar}" ] ; then + for f in "${HBASE_HOME}"/lib/hbase-backup*.jar; do + if [ -f "${f}" ]; then + CLASSPATH="${CLASSPATH}:${f}" + break + fi + done + fi +elif [ "$COMMAND" = "restore" ] ; then + CLASS='org.apache.hadoop.hbase.backup.RestoreDriver' + if [ -n "${shaded_jar}" ] ; then + for f in "${HBASE_HOME}"/lib/hbase-backup*.jar; do + if [ -f "${f}" ]; then + CLASSPATH="${CLASSPATH}:${f}" + break + fi + done + for f in "${HBASE_HOME}"/lib/commons-lang3*.jar; do + if [ -f "${f}" ]; then + CLASSPATH="${CLASSPATH}:${f}" + break + fi + done + fi elif [ "$COMMAND" = "regionsplitter" ] ; then CLASS='org.apache.hadoop.hbase.util.RegionSplitter' elif [ "$COMMAND" = "rowcounter" ] ; then diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml index c6049f744324..0981acf1a729 100644 --- a/hbase-assembly/pom.xml +++ b/hbase-assembly/pom.xml @@ -192,6 +192,10 @@ org.apache.hbase hbase-compression-zstd + + org.apache.hbase + hbase-backup + jline jline diff --git a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml index 61a2dfae8bb6..87164d0e73a3 100644 --- a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml +++ b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml @@ -57,6 +57,7 @@ org.apache.hbase:hbase-testing-util org.apache.hbase:hbase-thrift org.apache.hbase:hbase-zookeeper + org.apache.hbase:hbase-backup diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml index 53edfdfa7a2a..cb30a48f69a1 100644 --- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml +++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml @@ -59,6 +59,7 @@ org.apache.hbase:hbase-testing-util org.apache.hbase:hbase-thrift org.apache.hbase:hbase-zookeeper + org.apache.hbase:hbase-backup diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml new file mode 100644 index 000000000000..e12d699184f5 --- /dev/null +++ b/hbase-backup/pom.xml @@ -0,0 +1,307 @@ + + + + 4.0.0 + + org.apache.hbase + hbase-build-configuration + ${revision} + ../hbase-build-configuration + + hbase-backup + Apache HBase - Backup + Backup for HBase + + + + org.apache.hbase + hbase-annotations + test-jar + test + + + org.apache.hbase + hbase-logging + test-jar + test + + + org.apache.hbase + hbase-client + + + org.apache.hbase + hbase-server + + + javax.xml.bind + jaxb-api + + + javax.ws.rs + jsr311-api + + + + + org.apache.hbase + hbase-server + test-jar + test + + + org.apache.hbase + hbase-mapreduce + test-jar + test + + + org.apache.hbase + hbase-mapreduce + + + org.apache.hbase + hbase-common + + + org.apache.hbase + hbase-protocol-shaded + + + org.apache.hbase + hbase-testing-util + test + + + com.google.guava + guava + + + javax.xml.bind + jaxb-api + + + + + + org.apache.commons + commons-lang3 + + + org.slf4j + slf4j-api + + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + + + org.apache.zookeeper + zookeeper + + + org.slf4j + jcl-over-slf4j + test + + + org.slf4j + jul-to-slf4j + test + + + org.apache.logging.log4j + log4j-api + test + + + org.apache.logging.log4j + log4j-core + test + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + org.apache.logging.log4j + log4j-1.2-api + test + + + junit + junit + test + + + + + + + maven-assembly-plugin + + true + + + + + org.apache.maven.plugins + maven-source-plugin + + + net.revelc.code + warbucks-maven-plugin + + + + + + + hadoop-2.0 + + + + + !hadoop.profile + + + + + org.apache.hadoop + hadoop-distcp + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + io.netty + netty + ${netty.hadoop.version} + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + + build-classpath + + generate-test-resources + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + + hadoop-3.0 + + + hadoop.profile + 3.0 + + + + + org.apache.hadoop + hadoop-distcp + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + + build-classpath + + generate-test-resources + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + eclipse-specific + + + m2e.version + + + + + + + + org.eclipse.m2e + lifecycle-mapping + + + + + + + + + + + + diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java new file mode 100644 index 000000000000..25055fd5e8e6 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * The administrative API for HBase Backup. Construct an instance and call {@link #close()} + * afterwards. + *

+ * BackupAdmin can be used to create backups, restore data from backups and for other backup-related + * operations. + * @since 2.0 + */ +@InterfaceAudience.Private +public interface BackupAdmin extends Closeable { + + /** + * Backup given list of tables fully. This is a synchronous operation. It returns backup id on + * success or throw exception on failure. + * @param userRequest BackupRequest instance + * @return the backup Id + */ + + String backupTables(final BackupRequest userRequest) throws IOException; + + /** + * Restore backup + * @param request restore request + * @throws IOException exception + */ + void restore(RestoreRequest request) throws IOException; + + /** + * Describe backup image command + * @param backupId backup id + * @return backup info + * @throws IOException exception + */ + BackupInfo getBackupInfo(String backupId) throws IOException; + + /** + * Delete backup image command + * @param backupIds array of backup ids + * @return total number of deleted sessions + * @throws IOException exception + */ + int deleteBackups(String[] backupIds) throws IOException; + + /** + * Merge backup images command + * @param backupIds array of backup ids of images to be merged The resulting backup image will + * have the same backup id as the most recent image from a list of images to be + * merged + * @throws IOException exception + */ + void mergeBackups(String[] backupIds) throws IOException; + + /** + * Show backup history command + * @param n last n backup sessions + * @return list of backup info objects + * @throws IOException exception + */ + List getHistory(int n) throws IOException; + + /** + * Show backup history command with filters + * @param n last n backup sessions + * @param f list of filters + * @return list of backup info objects + * @throws IOException exception + */ + List getHistory(int n, BackupInfo.Filter... f) throws IOException; + + /** + * Backup sets list command - list all backup sets. Backup set is a named group of tables. + * @return all registered backup sets + * @throws IOException exception + */ + List listBackupSets() throws IOException; + + /** + * Backup set describe command. Shows list of tables in this particular backup set. + * @param name set name + * @return backup set description or null + * @throws IOException exception + */ + BackupSet getBackupSet(String name) throws IOException; + + /** + * Delete backup set command + * @param name backup set name + * @return true, if success, false - otherwise + * @throws IOException exception + */ + boolean deleteBackupSet(String name) throws IOException; + + /** + * Add tables to backup set command + * @param name name of backup set. + * @param tables array of tables to be added to this set. + * @throws IOException exception + */ + void addToBackupSet(String name, TableName[] tables) throws IOException; + + /** + * Remove tables from backup set + * @param name name of backup set. + * @param tables array of tables to be removed from this set. + * @throws IOException exception + */ + void removeFromBackupSet(String name, TableName[] tables) throws IOException; +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java new file mode 100644 index 000000000000..d710e82c4fd3 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java @@ -0,0 +1,56 @@ +/* + * 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.Configuration; +import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient; +import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class BackupClientFactory { + private BackupClientFactory() { + } + + public static TableBackupClient create(Connection conn, String backupId, BackupRequest request) + throws IOException { + Configuration conf = conn.getConfiguration(); + try { + String clsName = conf.get(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS); + if (clsName != null) { + Class clientImpl; + clientImpl = Class.forName(clsName).asSubclass(TableBackupClient.class); + TableBackupClient client = clientImpl.getDeclaredConstructor().newInstance(); + client.init(conn, backupId, request); + return client; + } + } catch (Exception e) { + throw new IOException(e); + } + + BackupType type = request.getBackupType(); + if (type == BackupType.FULL) { + return new FullTableBackupClient(conn, backupId, request); + } else { + return new IncrementalTableBackupClient(conn, backupId, request); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java new file mode 100644 index 000000000000..4753003fdf2b --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Backup copy job is a part of a backup process. The concrete implementation is responsible for + * copying data from a cluster to backup destination. Concrete implementation is provided by backup + * provider, see {@link BackupRestoreFactory} + */ +@InterfaceAudience.Private +public interface BackupCopyJob extends Configurable { + /** + * Copy backup data to destination + * @param backupInfo context object + * @param backupManager backup manager + * @param conf configuration + * @param backupType backup type (FULL or INCREMENTAL) + * @param options array of options (implementation-specific) + * @return result (0 - success, -1 failure ) + * @throws IOException exception + */ + int copy(BackupInfo backupInfo, BackupManager backupManager, Configuration conf, + BackupType backupType, String[] options) throws IOException; + + /** + * Cancel copy job + * @param jobHandler backup copy job handler + * @throws IOException if cancelling the jobs fails + */ + void cancel(String jobHandler) throws IOException; +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java new file mode 100644 index 000000000000..547a39c8d623 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java @@ -0,0 +1,211 @@ +/* + * 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_BACKUP_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_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_KEEP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_LIST; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_DESC; + +import java.io.IOException; +import java.net.URI; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand; +import org.apache.hadoop.hbase.backup.impl.BackupCommands; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.logging.Log4jUtils; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; + +/** + * Command-line entry point for backup operation + */ +@InterfaceAudience.Private +public class BackupDriver extends AbstractHBaseTool { + + private static final Logger LOG = LoggerFactory.getLogger(BackupDriver.class); + private CommandLine cmd; + + public BackupDriver() throws IOException { + init(); + } + + protected void init() throws IOException { + // disable irrelevant loggers to avoid it mess up command output + Log4jUtils.disableZkAndClientLoggers(); + } + + private int parseAndRun(String[] args) throws IOException { + + // Check if backup is enabled + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + + String cmd = null; + String[] remainArgs = null; + if (args == null || args.length == 0) { + printToolUsage(); + return -1; + } else { + cmd = args[0]; + remainArgs = new String[args.length - 1]; + if (args.length > 1) { + System.arraycopy(args, 1, remainArgs, 0, args.length - 1); + } + } + + BackupCommand type = BackupCommand.HELP; + if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.CREATE; + } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.HELP; + } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.DELETE; + } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.DESCRIBE; + } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.HISTORY; + } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.PROGRESS; + } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.SET; + } else if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.REPAIR; + } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.MERGE; + } else { + System.out.println("Unsupported command for backup: " + cmd); + printToolUsage(); + return -1; + } + + // enable debug logging + if (this.cmd.hasOption(OPTION_DEBUG)) { + Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "DEBUG"); + } + + BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd); + if (type == BackupCommand.CREATE && conf != null) { + ((BackupCommands.CreateCommand) command).setConf(conf); + } + try { + command.execute(); + } catch (IOException e) { + if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) { + return -1; + } + throw e; + } finally { + command.finish(); + } + return 0; + } + + @Override + protected void addOptions() { + // define supported options + addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); + addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC); + addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC); + addOptWithArg(OPTION_LIST, OPTION_BACKUP_LIST_DESC); + addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC); + addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC); + addOptWithArg(OPTION_SET, OPTION_SET_DESC); + addOptWithArg(OPTION_PATH, OPTION_PATH_DESC); + addOptWithArg(OPTION_KEEP, OPTION_KEEP_DESC); + addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_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 = CommonFSUtils.getRootDir(conf); + URI defaultFs = hbasedir.getFileSystem(conf).getUri(); + CommonFSUtils.setFsDefault(conf, new Path(defaultFs)); + int ret = ToolRunner.run(conf, new BackupDriver(), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws IOException { + Objects.requireNonNull(conf, "Tool configuration is not initialized"); + + CommandLine cmd; + try { + // parse the command line arguments + cmd = parseArgs(args); + cmdLineArgs = args; + } catch (Exception e) { + System.err.println("Error when parsing command-line arguments: " + e.getMessage()); + 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(BackupCommands.USAGE); + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java new file mode 100644 index 000000000000..619cecaeaaac --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java @@ -0,0 +1,180 @@ +/* + * 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.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; + +/** + * Implementation of a file cleaner that checks if an hfile is still referenced by backup before + * deleting it from hfile archive directory. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abortable { + private static final Logger LOG = LoggerFactory.getLogger(BackupHFileCleaner.class); + private boolean stopped = false; + private boolean aborted; + private Configuration conf; + private Connection connection; + private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table + secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table + // used by unit test to skip reading backup:system + private boolean checkForFullyBackedUpTables = true; + private List fullyBackedUpTables = null; + + private Set getFilenameFromBulkLoad(Map>[] maps) { + Set filenames = new HashSet<>(); + for (Map> map : maps) { + if (map == null) { + continue; + } + + for (List paths : map.values()) { + for (Path p : paths) { + filenames.add(p.getName()); + } + } + } + return filenames; + } + + private Set loadHFileRefs(List tableList) throws IOException { + if (connection == null) { + connection = ConnectionFactory.createConnection(conf); + } + try (BackupSystemTable tbl = new BackupSystemTable(connection)) { + Map>[] res = tbl.readBulkLoadedFiles(null, tableList); + secondPrevReadFromBackupTbl = prevReadFromBackupTbl; + prevReadFromBackupTbl = EnvironmentEdgeManager.currentTime(); + return getFilenameFromBulkLoad(res); + } + } + + @InterfaceAudience.Private + void setCheckForFullyBackedUpTables(boolean b) { + checkForFullyBackedUpTables = b; + } + + @Override + public Iterable getDeletableFiles(Iterable files) { + if (conf == null) { + return files; + } + // obtain the Set of TableName's which have been fully backed up + // so that we filter BulkLoad to be returned from server + if (checkForFullyBackedUpTables) { + if (connection == null) { + return files; + } + + try (BackupSystemTable tbl = new BackupSystemTable(connection)) { + fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL); + } catch (IOException ioe) { + LOG.error("Failed to get tables which have been fully backed up, skipping checking", ioe); + return Collections.emptyList(); + } + Collections.sort(fullyBackedUpTables); + } + final Set hfileRefs; + try { + hfileRefs = loadHFileRefs(fullyBackedUpTables); + } catch (IOException ioe) { + LOG.error("Failed to read hfile references, skipping checking deletable files", ioe); + return Collections.emptyList(); + } + Iterable deletables = Iterables.filter(files, file -> { + // If the file is recent, be conservative and wait for one more scan of backup:system table + if (file.getModificationTime() > secondPrevReadFromBackupTbl) { + return false; + } + String hfile = file.getPath().getName(); + boolean foundHFileRef = hfileRefs.contains(hfile); + return !foundHFileRef; + }); + return deletables; + } + + @Override + public boolean isFileDeletable(FileStatus fStat) { + // work is done in getDeletableFiles() + return true; + } + + @Override + public void setConf(Configuration config) { + this.conf = config; + this.connection = null; + try { + this.connection = ConnectionFactory.createConnection(conf); + } catch (IOException ioe) { + LOG.error("Couldn't establish connection", ioe); + } + } + + @Override + public void stop(String why) { + if (this.stopped) { + return; + } + if (this.connection != null) { + try { + this.connection.close(); + } catch (IOException ioe) { + LOG.debug("Got " + ioe + " when closing connection"); + } + } + this.stopped = true; + } + + @Override + public boolean isStopped() { + return this.stopped; + } + + @Override + public void abort(String why, Throwable e) { + LOG.warn("Aborting ReplicationHFileCleaner because " + why, e); + this.aborted = true; + stop(why); + } + + @Override + public boolean isAborted() { + return this.aborted; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java new file mode 100644 index 000000000000..fdad0d549830 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -0,0 +1,578 @@ +/* + * 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.lang3.StringUtils; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; + +/** + * An object to encapsulate the information for each backup session + */ +@InterfaceAudience.Private +public class BackupInfo implements Comparable { + private static final Logger LOG = LoggerFactory.getLogger(BackupInfo.class); + + public interface Filter { + /** + * Filter interface + * @param info backup info + * @return true if info passes filter, false otherwise + */ + boolean apply(BackupInfo info); + } + + /** + * Backup session states + */ + public enum BackupState { + RUNNING, + COMPLETE, + FAILED, + ANY + } + + /** + * BackupPhase - phases of an ACTIVE backup session (running), when state of a backup session is + * BackupState.RUNNING + */ + public 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 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 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 Map> tableSetTimestampMap; + + /** + * Previous Region server log timestamps for table set after distributed log roll key - table + * name, value - map of RegionServer hostname -> last log rolled timestamp + */ + private Map> incrTimestampMap; + + /** + * 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<>(); + } + + 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 backupTableInfoMap) { + this.backupTableInfoMap = backupTableInfoMap; + } + + public Map> getTableSetTimestampMap() { + return tableSetTimestampMap; + } + + public void setTableSetTimestampMap(Map> 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 getSnapshotNames() { + List snapshotNames = new ArrayList<>(); + for (BackupTableInfo backupStatus : this.backupTableInfoMap.values()) { + snapshotNames.add(backupStatus.getSnapshotName()); + } + return snapshotNames; + } + + public Set getTables() { + return this.backupTableInfoMap.keySet(); + } + + public List getTableNames() { + return new ArrayList<>(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 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 getIncrBackupFileList() { + return incrBackupFileList; + } + + public void setIncrBackupFileList(List incrBackupFileList) { + this.incrBackupFileList = incrBackupFileList; + } + + /** + * Set the new region server log timestamps after distributed log roll + * @param prevTableSetTimestampMap table timestamp map + */ + public void setIncrTimestampMap(Map> prevTableSetTimestampMap) { + this.incrTimestampMap = prevTableSetTimestampMap; + } + + /** + * Get new region server log timestamps after distributed log roll + * @return new region server log timestamps + */ + public Map> getIncrTimestampMap() { + return this.incrTimestampMap; + } + + public TableName getTableBySnapshot(String snapshotName) { + for (Entry 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); + setTableSetTimestampMap(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.toString(), e); + return false; + } + } else { + return false; + } + } + + @Override + public String toString() { + return backupId; + } + + public byte[] toByteArray() throws IOException { + return toProtosBackupInfo().toByteArray(); + } + + private void setBackupTableInfoMap(BackupProtos.BackupInfo.Builder builder) { + for (Entry entry : backupTableInfoMap.entrySet()) { + builder.addBackupTableInfo(entry.getValue().toProto()); + } + } + + private void setTableSetTimestampMap(BackupProtos.BackupInfo.Builder builder) { + if (this.getTableSetTimestampMap() != null) { + for (Entry> entry : this.getTableSetTimestampMap().entrySet()) { + builder.putTableSetTimestamp(entry.getKey().getNameAsString(), + BackupProtos.BackupInfo.RSTimestampMap.newBuilder().putAllRsTimestamp(entry.getValue()) + .build()); + } + } + } + + 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.setTableSetTimestampMap(getTableSetTimestampMap(proto.getTableSetTimestampMap())); + 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 toMap(List list) { + HashMap map = new HashMap<>(); + for (BackupProtos.BackupTableInfo tbs : list) { + map.put(ProtobufUtil.toTableName(tbs.getTableName()), BackupTableInfo.convert(tbs)); + } + return map; + } + + private static Map> + getTableSetTimestampMap(Map map) { + Map> tableSetTimestampMap = new HashMap<>(); + for (Entry entry : map.entrySet()) { + tableSetTimestampMap.put(TableName.valueOf(entry.getKey()), + entry.getValue().getRsTimestampMap()); + } + + return tableSetTimestampMap; + } + + 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(","); + Calendar cal = Calendar.getInstance(); + cal.setTimeInMillis(getStartTs()); + Date 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() { + StringBuilder sb = new StringBuilder(); + 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); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java new file mode 100644 index 000000000000..1e2b17145025 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java @@ -0,0 +1,37 @@ +/* + * 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.yetus.audience.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 if the backup merge operation fails + */ + void run(String[] backupIds) throws IOException; +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java new file mode 100644 index 000000000000..73f97365adbe --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java @@ -0,0 +1,107 @@ +/* + * 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 java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +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.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An Observer to facilitate backup operations + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class BackupObserver implements RegionCoprocessor, RegionObserver { + private static final Logger LOG = LoggerFactory.getLogger(BackupObserver.class); + + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + @Override + public void postBulkLoadHFile(ObserverContext ctx, + List> stagingFamilyPaths, Map> finalPaths) + throws IOException { + Configuration cfg = ctx.getEnvironment().getConfiguration(); + if (finalPaths == null) { + // there is no need to record state + return; + } + if (!BackupManager.isBackupEnabled(cfg)) { + LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled"); + return; + } + try (Connection connection = ConnectionFactory.createConnection(cfg); + BackupSystemTable tbl = new BackupSystemTable(connection)) { + List fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL); + RegionInfo 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.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths); + } catch (IOException ioe) { + LOG.error("Failed to get tables which have been fully backed up", ioe); + } + } + + @Override + public void preCommitStoreFile(final ObserverContext ctx, + final byte[] family, final List> 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 fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL); + RegionInfo 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; + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java new file mode 100644 index 000000000000..c9c7a5b61810 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java @@ -0,0 +1,151 @@ +/* + * 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.yetus.audience.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 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 Builder withYarnPoolName(String name) { + request.setYarnPoolName(name); + return this; + } + + public BackupRequest build() { + return request; + } + + } + + private BackupType type; + private List tableList; + private String targetRootDir; + private int totalTasks = -1; + private long bandwidth = -1L; + private String backupSetName; + private String yarnPoolName; + + private BackupRequest() { + } + + private BackupRequest setBackupType(BackupType type) { + this.type = type; + return this; + } + + public BackupType getBackupType() { + return this.type; + } + + private BackupRequest setTableList(List tableList) { + this.tableList = tableList; + return this; + } + + public List 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; + } + + public String getYarnPoolName() { + return yarnPoolName; + } + + public void setYarnPoolName(String yarnPoolName) { + this.yarnPoolName = yarnPoolName; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java new file mode 100644 index 000000000000..56c454519d81 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -0,0 +1,140 @@ +/* + * 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.yetus.audience.InterfaceAudience; + +/** + * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants + */ +@InterfaceAudience.Private +public interface BackupRestoreConstants { + /* + * Backup/Restore constants + */ + String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name"; + String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "backup:system"; + + String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl"; + + int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER; + String BACKUP_ENABLE_KEY = "hbase.backup.enable"; + boolean BACKUP_ENABLE_DEFAULT = false; + + String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max"; + int DEFAULT_BACKUP_MAX_ATTEMPTS = 10; + + String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms"; + int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000; + + /* + * Drivers option list + */ + String OPTION_OVERWRITE = "o"; + String OPTION_OVERWRITE_DESC = "Overwrite data if any of the restore target tables exists"; + + String OPTION_CHECK = "c"; + String OPTION_CHECK_DESC = + "Check restore sequence and dependencies only (does not execute the command)"; + + String OPTION_SET = "s"; + String OPTION_SET_DESC = "Backup set name"; + String OPTION_SET_RESTORE_DESC = "Backup set to restore, mutually exclusive with -t (table list)"; + String OPTION_SET_BACKUP_DESC = "Backup set to backup, mutually exclusive with -t (table list)"; + String OPTION_DEBUG = "d"; + String OPTION_DEBUG_DESC = "Enable debug loggings"; + + String OPTION_TABLE = "t"; + String OPTION_TABLE_DESC = + "Table name. If specified, only backup images," + " which contain this table will be listed."; + + String OPTION_LIST = "l"; + String OPTION_TABLE_LIST_DESC = "Table name list, comma-separated."; + String OPTION_BACKUP_LIST_DESC = "Backup ids list, comma-separated."; + + String OPTION_BANDWIDTH = "b"; + String OPTION_BANDWIDTH_DESC = "Bandwidth per task (MapReduce task) in MB/s"; + + String OPTION_WORKERS = "w"; + String OPTION_WORKERS_DESC = "Number of parallel MapReduce tasks to execute"; + + String OPTION_RECORD_NUMBER = "n"; + String OPTION_RECORD_NUMBER_DESC = "Number of records of backup history. Default: 10"; + + String OPTION_PATH = "p"; + String OPTION_PATH_DESC = "Backup destination root directory path"; + + String OPTION_KEEP = "k"; + String OPTION_KEEP_DESC = "Specifies maximum age of backup (in days) to keep during bulk delete"; + + String OPTION_TABLE_MAPPING = "m"; + String OPTION_TABLE_MAPPING_DESC = "A comma separated list of target tables. " + + "If specified, each table in must have a mapping"; + String OPTION_YARN_QUEUE_NAME = "q"; + String OPTION_YARN_QUEUE_NAME_DESC = "Yarn queue name to run backup create command on"; + String OPTION_YARN_QUEUE_NAME_RESTORE_DESC = "Yarn queue name to run backup restore command on"; + + String JOB_NAME_CONF_KEY = "mapreduce.job.name"; + + 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" + + "hbase.coprocessor.region.classes=YOUR_CLASSES," + + "org.apache.hadoop.hbase.backup.BackupObserver\n" + "and restart the cluster\n" + + "For more information please see http://hbase.apache.org/book.html#backuprestore\n"; + String ENABLE_BACKUP = "Backup is not enabled. To enable backup, " + "in hbase-site.xml, set:\n " + + BACKUP_CONFIG_STRING; + + String VERIFY_BACKUP = "To enable backup, in hbase-site.xml, set:\n " + BACKUP_CONFIG_STRING; + + /* + * Delimiter in table name list in restore command + */ + String TABLENAME_DELIMITER_IN_COMMAND = ","; + + String CONF_STAGING_ROOT = "snapshot.export.staging.root"; + + String BACKUPID_PREFIX = "backup_"; + + 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 + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java new file mode 100644 index 000000000000..40bbb4bc7fea --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java @@ -0,0 +1,78 @@ +/* + * 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.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * 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 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 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 cls = conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, + MapReduceBackupMergeJob.class, BackupMergeJob.class); + BackupMergeJob service = ReflectionUtils.newInstance(cls, conf); + service.setConf(conf); + return service; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java new file mode 100644 index 000000000000..01097422e3a1 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java @@ -0,0 +1,79 @@ +/* + * 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.yetus.audience.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(ProtobufUtil.toProtoTableName(table)); + return builder.build(); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java new file mode 100644 index 000000000000..c41a4a182435 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -0,0 +1,158 @@ +/* + * 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.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.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 final class HBackupFileSystem { + public static final Logger LOG = LoggerFactory.getLogger(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; + } + + /** + * Get backup temporary directory + * @param backupRootDir backup root + * @return backup tmp directory path + */ + public static Path getBackupTmpDirPath(String backupRootDir) { + return new Path(backupRootDir, ".tmp"); + } + + /** + * Get backup tmp directory for backupId + * @param backupRoot backup root + * @param backupId backup id + * @return backup tmp directory path + */ + public static Path getBackupTmpDirPathForBackupId(String backupRoot, String backupId) { + return new Path(getBackupTmpDirPath(backupRoot), backupId); + } + + 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 { + FileSystem fs = backupRootPath.getFileSystem(conf); + Path 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 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); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java new file mode 100644 index 000000000000..cb01469c8f18 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -0,0 +1,270 @@ +/* + * 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 static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_RESTORE_DESC; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Objects; +import org.apache.commons.lang3.StringUtils; +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.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.logging.Log4jUtils; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; + +/** + * Command-line entry point for restore operation + */ +@InterfaceAudience.Private +public class RestoreDriver extends AbstractHBaseTool { + private static final Logger LOG = LoggerFactory.getLogger(RestoreDriver.class); + private CommandLine cmd; + + private static final String USAGE_STRING = + "Usage: hbase restore [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() { + // disable irrelevant loggers to avoid it mess up command output + Log4jUtils.disableZkAndClientLoggers(); + } + + private int parseAndRun() throws IOException { + // Check if backup is enabled + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + + // enable debug logging + if (cmd.hasOption(OPTION_DEBUG)) { + Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "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; + } + + if (cmd.hasOption(OPTION_YARN_QUEUE_NAME)) { + String queueName = cmd.getOptionValue(OPTION_YARN_QUEUE_NAME); + // Set system property value for MR job + System.setProperty("mapreduce.job.queuename", queueName); + } + + // 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; + 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); + } 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) { + LOG.error("Error while running restore backup", e); + return -5; + } + return 0; + } + + private String getTablesForSet(Connection conn, String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List tables = table.describeBackupSet(name); + + if (tables == null) { + return null; + } + + return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } + + @Override + protected void addOptions() { + // define supported options + addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC); + addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC); + addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); + addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC); + addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC); + addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC); + addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_RESTORE_DESC); + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + return parseAndRun(); + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + Path hbasedir = CommonFSUtils.getRootDir(conf); + URI defaultFs = hbasedir.getFileSystem(conf).getUri(); + CommonFSUtils.setFsDefault(conf, new Path(defaultFs)); + int ret = ToolRunner.run(conf, new RestoreDriver(), args); + System.exit(ret); + } + + @Override + public int run(String[] args) { + Objects.requireNonNull(conf, "Tool configuration is not initialized"); + + 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() { + 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); + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java new file mode 100644 index 000000000000..b014e6693bbc --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java @@ -0,0 +1,43 @@ +/* + * 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.yetus.audience.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 if running the job fails + */ + void run(Path[] dirPaths, TableName[] fromTables, TableName[] toTables, boolean fullBackupRestore) + throws IOException; +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java new file mode 100644 index 000000000000..eb4786f57869 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java @@ -0,0 +1,133 @@ +/* + * 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.yetus.audience.InterfaceAudience; + +/** + * POJO class for restore request + */ +@InterfaceAudience.Private +public final 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; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java new file mode 100644 index 000000000000..f580fb0c47bb --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -0,0 +1,716 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupClientFactory; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupMergeJob; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +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.RestoreRequest; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@InterfaceAudience.Private +public class BackupAdminImpl implements BackupAdmin { + public final static String CHECK_OK = "Checking backup images: OK"; + public final static String CHECK_FAILED = + "Checking backup images: Failed. Some dependencies are missing for restore"; + private static final Logger LOG = LoggerFactory.getLogger(BackupAdminImpl.class); + + private final Connection conn; + + public BackupAdminImpl(Connection conn) { + this.conn = conn; + } + + @Override + public void close() { + } + + @Override + public BackupInfo getBackupInfo(String backupId) throws IOException { + BackupInfo backupInfo; + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (backupId == null) { + ArrayList recentSessions = table.getBackupInfos(BackupState.RUNNING); + if (recentSessions.isEmpty()) { + LOG.warn("No ongoing sessions found."); + return null; + } + // else show status for ongoing session + // must be one maximum + return recentSessions.get(0); + } else { + backupInfo = table.readBackupInfo(backupId); + return backupInfo; + } + } + } + + @Override + public int deleteBackups(String[] backupIds) throws IOException { + + int totalDeleted = 0; + Map> allTablesMap = new HashMap<>(); + + boolean deleteSessionStarted; + boolean snapshotDone; + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + // Step 1: Make sure there is no active session + // is running by using startBackupSession API + // If there is an active session in progress, exception will be thrown + try { + sysTable.startBackupExclusiveOperation(); + deleteSessionStarted = true; + } catch (IOException e) { + LOG.warn("You can not run delete command while active backup session is in progress. \n" + + "If there is no active backup session running, run backup repair utility to " + + "restore \nbackup system integrity."); + return -1; + } + + // Step 2: Make sure there is no failed session + List list = sysTable.getBackupInfos(BackupState.RUNNING); + if (list.size() != 0) { + // ailed sessions found + LOG.warn("Failed backup session found. Run backup repair tool first."); + return -1; + } + + // Step 3: Record delete session + sysTable.startDeleteOperation(backupIds); + // Step 4: Snapshot backup system table + if (!BackupSystemTable.snapshotExists(conn)) { + BackupSystemTable.snapshot(conn); + } else { + LOG.warn("Backup system table snapshot exists"); + } + snapshotDone = true; + try { + for (int i = 0; i < backupIds.length; i++) { + BackupInfo info = sysTable.readBackupInfo(backupIds[i]); + if (info != null) { + String rootDir = info.getBackupRootDir(); + HashSet allTables = allTablesMap.get(rootDir); + if (allTables == null) { + allTables = new HashSet<>(); + allTablesMap.put(rootDir, allTables); + } + allTables.addAll(info.getTableNames()); + totalDeleted += deleteBackup(backupIds[i], sysTable); + } + } + finalizeDelete(allTablesMap, sysTable); + // Finish + sysTable.finishDeleteOperation(); + // delete snapshot + BackupSystemTable.deleteSnapshot(conn); + } catch (IOException e) { + // Fail delete operation + // Step 1 + if (snapshotDone) { + if (BackupSystemTable.snapshotExists(conn)) { + BackupSystemTable.restoreFromSnapshot(conn); + // delete snapshot + BackupSystemTable.deleteSnapshot(conn); + // We still have record with unfinished delete operation + LOG.error("Delete operation failed, please run backup repair utility to restore " + + "backup system integrity", e); + throw e; + } else { + LOG.warn("Delete operation succeeded, there were some errors: ", e); + } + } + + } finally { + if (deleteSessionStarted) { + sysTable.finishBackupExclusiveOperation(); + } + } + } + return totalDeleted; + } + + /** + * Updates incremental backup set for every backupRoot + * @param tablesMap map [backupRoot: {@code Set}] + * @param table backup system table + * @throws IOException if a table operation fails + */ + private void finalizeDelete(Map> tablesMap, BackupSystemTable table) + throws IOException { + for (String backupRoot : tablesMap.keySet()) { + Set incrTableSet = table.getIncrementalBackupTableSet(backupRoot); + Map> tableMap = + table.getBackupHistoryForTableSet(incrTableSet, backupRoot); + for (Map.Entry> entry : tableMap.entrySet()) { + if (entry.getValue() == null) { + // No more backups for a table + incrTableSet.remove(entry.getKey()); + } + } + if (!incrTableSet.isEmpty()) { + table.addIncrementalBackupTableSet(incrTableSet, backupRoot); + } else { // empty + table.deleteIncrementalBackupTableSet(backupRoot); + } + } + } + + /** + * Delete single backup and all related backups
+ * Algorithm:
+ * Backup type: FULL or INCREMENTAL
+ * Is this last backup session for table T: YES or NO
+ * For every table T from table list 'tables':
+ * if(FULL, YES) deletes only physical data (PD)
+ * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,
+ * until we either reach the most recent backup for T in the system or FULL backup
+ * which includes T
+ * if(INCREMENTAL, YES) deletes only physical data (PD) if(INCREMENTAL, NO) deletes physical data + * and for table T scans all backup images between last
+ * FULL backup, which is older than the backup being deleted and the next FULL backup (if exists) + *
+ * or last one for a particular table T and removes T from list of backup tables. + * @param backupId backup id + * @param sysTable backup system table + * @return total number of deleted backup images + * @throws IOException if deleting the backup fails + */ + private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { + BackupInfo backupInfo = sysTable.readBackupInfo(backupId); + + int totalDeleted = 0; + if (backupInfo != null) { + LOG.info("Deleting backup " + backupInfo.getBackupId() + " ..."); + // Step 1: clean up data for backup session (idempotent) + BackupUtils.cleanupBackupData(backupInfo, conn.getConfiguration()); + // List of tables in this backup; + List tables = backupInfo.getTableNames(); + long startTime = backupInfo.getStartTs(); + for (TableName tn : tables) { + boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime); + if (isLastBackupSession) { + continue; + } + // else + List affectedBackups = getAffectedBackupSessions(backupInfo, tn, sysTable); + for (BackupInfo info : affectedBackups) { + if (info.equals(backupInfo)) { + continue; + } + removeTableFromBackupImage(info, tn, sysTable); + } + } + Map map = sysTable.readBulkLoadedFiles(backupId); + FileSystem fs = FileSystem.get(conn.getConfiguration()); + boolean success = true; + int numDeleted = 0; + for (String f : map.values()) { + Path p = new Path(f); + try { + LOG.debug("Delete backup info " + p + " for " + backupInfo.getBackupId()); + if (!fs.delete(p)) { + if (fs.exists(p)) { + LOG.warn(f + " was not deleted"); + success = false; + } + } else { + numDeleted++; + } + } catch (IOException ioe) { + LOG.warn(f + " was not deleted", ioe); + success = false; + } + } + if (LOG.isDebugEnabled()) { + LOG.debug(numDeleted + " bulk loaded files out of " + map.size() + " were deleted"); + } + if (success) { + sysTable.deleteBulkLoadedRows(new ArrayList<>(map.keySet())); + } + + sysTable.deleteBackupInfo(backupInfo.getBackupId()); + LOG.info("Delete backup " + backupInfo.getBackupId() + " completed."); + totalDeleted++; + } else { + LOG.warn("Delete backup failed: no information found for backupID=" + backupId); + } + return totalDeleted; + } + + private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable) + throws IOException { + List tables = info.getTableNames(); + LOG.debug( + "Remove " + tn + " from " + info.getBackupId() + " tables=" + info.getTableListAsString()); + if (tables.contains(tn)) { + tables.remove(tn); + + if (tables.isEmpty()) { + LOG.debug("Delete backup info " + info.getBackupId()); + + sysTable.deleteBackupInfo(info.getBackupId()); + // Idempotent operation + BackupUtils.cleanupBackupData(info, conn.getConfiguration()); + } else { + info.setTables(tables); + sysTable.updateBackupInfo(info); + // Now, clean up directory for table (idempotent) + cleanupBackupDir(info, tn, conn.getConfiguration()); + } + } + } + + private List getAffectedBackupSessions(BackupInfo backupInfo, TableName tn, + BackupSystemTable table) throws IOException { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn); + long ts = backupInfo.getStartTs(); + List list = new ArrayList<>(); + List history = table.getBackupHistory(backupInfo.getBackupRootDir()); + // Scan from most recent to backupInfo + // break when backupInfo reached + for (BackupInfo info : history) { + if (info.getStartTs() == ts) { + break; + } + List tables = info.getTableNames(); + if (tables.contains(tn)) { + BackupType bt = info.getType(); + if (bt == BackupType.FULL) { + // Clear list if we encounter FULL backup + list.clear(); + } else { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn + + " added " + info.getBackupId() + " tables=" + info.getTableListAsString()); + list.add(info); + } + } + } + return list; + } + + /** + * Clean up the data at target directory + * @throws IOException if cleaning up the backup directory fails + */ + private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) + throws IOException { + try { + // clean up the data at target directory + String targetDir = backupInfo.getBackupRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupInfo.getBackupId()); + return; + } + + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + Path targetDirPath = new Path(BackupUtils.getTableBackupDir(backupInfo.getBackupRootDir(), + backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table + + "at " + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + throw e1; + } + } + + private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) + throws IOException { + List history = table.getBackupHistory(); + for (BackupInfo info : history) { + List tables = info.getTableNames(); + if (!tables.contains(tn)) { + continue; + } + return info.getStartTs() <= startTime; + } + return false; + } + + @Override + public List getHistory(int n) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + + if (history.size() <= n) { + return history; + } + + List list = new ArrayList<>(); + for (int i = 0; i < n; i++) { + list.add(history.get(i)); + } + return list; + } + } + + @Override + public List getHistory(int n, BackupInfo.Filter... filters) throws IOException { + if (filters.length == 0) { + return getHistory(n); + } + + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + List result = new ArrayList<>(); + for (BackupInfo bi : history) { + if (result.size() == n) { + break; + } + + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(bi)) { + passed = false; + break; + } + } + if (passed) { + result.add(bi); + } + } + return result; + } + } + + @Override + public List listBackupSets() throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.listBackupSets(); + List bslist = new ArrayList<>(); + for (String s : list) { + List tables = table.describeBackupSet(s); + if (tables != null) { + bslist.add(new BackupSet(s, tables)); + } + } + return bslist; + } + } + + @Override + public BackupSet getBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.describeBackupSet(name); + + if (list == null) { + return null; + } + + return new BackupSet(name, list); + } + } + + @Override + public boolean deleteBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (table.describeBackupSet(name) == null) { + return false; + } + table.deleteBackupSet(name); + return true; + } + } + + @Override + public void addToBackupSet(String name, TableName[] tables) throws IOException { + String[] tableNames = new String[tables.length]; + try (final BackupSystemTable table = new BackupSystemTable(conn); + final Admin admin = conn.getAdmin()) { + for (int i = 0; i < tables.length; i++) { + tableNames[i] = tables[i].getNameAsString(); + if (!admin.tableExists(TableName.valueOf(tableNames[i]))) { + throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist"); + } + } + table.addToBackupSet(name, tableNames); + LOG.info( + "Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name + "' backup set"); + } + } + + @Override + public void removeFromBackupSet(String name, TableName[] tables) throws IOException { + LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'"); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + table.removeFromBackupSet(name, toStringArray(tables)); + LOG.info( + "Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "' completed."); + } + } + + private String[] toStringArray(TableName[] list) { + String[] arr = new String[list.length]; + for (int i = 0; i < list.length; i++) { + arr[i] = list[i].toString(); + } + return arr; + } + + @Override + public void restore(RestoreRequest request) throws IOException { + if (request.isCheck()) { + HashMap backupManifestMap = new HashMap<>(); + // check and load backup image manifest for the tables + Path rootPath = new Path(request.getBackupRootDir()); + String backupId = request.getBackupId(); + TableName[] sTableArray = request.getFromTables(); + HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, + conn.getConfiguration(), rootPath, backupId); + + // Check and validate the backup image and its dependencies + if (BackupUtils.validate(backupManifestMap, conn.getConfiguration())) { + LOG.info(CHECK_OK); + } else { + LOG.error(CHECK_FAILED); + } + return; + } + // Execute restore request + new RestoreTablesClient(conn, request).execute(); + } + + @Override + public String backupTables(BackupRequest request) throws IOException { + BackupType type = request.getBackupType(); + String targetRootDir = request.getTargetRootDir(); + List tableList = request.getTableList(); + + String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime(); + if (type == BackupType.INCREMENTAL) { + Set incrTableSet; + try (BackupSystemTable table = new BackupSystemTable(conn)) { + incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + } + + if (incrTableSet.isEmpty()) { + String msg = + "Incremental backup table set contains no tables. " + "You need to run full backup first " + + (tableList != null ? "on " + StringUtils.join(tableList, ",") : ""); + + throw new IOException(msg); + } + if (tableList != null) { + tableList.removeAll(incrTableSet); + if (!tableList.isEmpty()) { + String extraTables = StringUtils.join(tableList, ","); + String msg = "Some tables (" + extraTables + ") haven't gone through full backup. " + + "Perform full backup on " + extraTables + " first, " + "then retry the command"; + throw new IOException(msg); + } + } + tableList = Lists.newArrayList(incrTableSet); + } + if (tableList != null && !tableList.isEmpty()) { + for (TableName table : tableList) { + String targetTableBackupDir = + HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); + Path targetTableBackupDirPath = new Path(targetTableBackupDir); + FileSystem outputFs = + FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration()); + if (outputFs.exists(targetTableBackupDirPath)) { + throw new IOException( + "Target backup directory " + targetTableBackupDir + " exists already."); + } + outputFs.mkdirs(targetTableBackupDirPath); + } + ArrayList nonExistingTableList = null; + try (Admin admin = conn.getAdmin()) { + for (TableName tableName : tableList) { + if (!admin.tableExists(tableName)) { + if (nonExistingTableList == null) { + nonExistingTableList = new ArrayList<>(); + } + nonExistingTableList.add(tableName); + } + } + } + if (nonExistingTableList != null) { + if (type == BackupType.INCREMENTAL) { + // Update incremental backup set + tableList = excludeNonExistingTables(tableList, nonExistingTableList); + } else { + // Throw exception only in full mode - we try to backup non-existing table + throw new IOException( + "Non-existing tables found in the table list: " + nonExistingTableList); + } + } + } + + // update table list + BackupRequest.Builder builder = new BackupRequest.Builder(); + request = builder.withBackupType(request.getBackupType()).withTableList(tableList) + .withTargetRootDir(request.getTargetRootDir()).withBackupSetName(request.getBackupSetName()) + .withTotalTasks(request.getTotalTasks()).withBandwidthPerTasks((int) request.getBandwidth()) + .build(); + + TableBackupClient client; + try { + client = BackupClientFactory.create(conn, backupId, request); + } catch (IOException e) { + LOG.error("There is an active session already running"); + throw e; + } + + client.execute(); + + return backupId; + } + + private List excludeNonExistingTables(List tableList, + List nonExistingTableList) { + for (TableName table : nonExistingTableList) { + tableList.remove(table); + } + return tableList; + } + + @Override + public void mergeBackups(String[] backupIds) throws IOException { + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + checkIfValidForMerge(backupIds, sysTable); + // TODO run job on remote cluster + BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration()); + job.run(backupIds); + } + } + + /** + * Verifies that backup images are valid for merge. + *

    + *
  • All backups MUST be in the same destination + *
  • No FULL backups are allowed - only INCREMENTAL + *
  • All backups must be in COMPLETE state + *
  • No holes in backup list are allowed + *
+ *

+ * @param backupIds list of backup ids + * @param table backup system table + * @throws IOException if the backup image is not valid for merge + */ + private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) + throws IOException { + String backupRoot = null; + + final Set allTables = new HashSet<>(); + final Set allBackups = new HashSet<>(); + long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE; + for (String backupId : backupIds) { + BackupInfo bInfo = table.readBackupInfo(backupId); + if (bInfo == null) { + String msg = "Backup session " + backupId + " not found"; + throw new IOException(msg); + } + if (backupRoot == null) { + backupRoot = bInfo.getBackupRootDir(); + } else if (!bInfo.getBackupRootDir().equals(backupRoot)) { + throw new IOException("Found different backup destinations in a list of a backup sessions " + + "\n1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir()); + } + if (bInfo.getType() == BackupType.FULL) { + throw new IOException("FULL backup image can not be merged for: \n" + bInfo); + } + + if (bInfo.getState() != BackupState.COMPLETE) { + throw new IOException("Backup image " + backupId + + " can not be merged becuase of its state: " + bInfo.getState()); + } + allBackups.add(backupId); + allTables.addAll(bInfo.getTableNames()); + long time = bInfo.getStartTs(); + if (time < minTime) { + minTime = time; + } + if (time > maxTime) { + maxTime = time; + } + } + + final long startRangeTime = minTime; + final long endRangeTime = maxTime; + final String backupDest = backupRoot; + // Check we have no 'holes' in backup id list + // Filter 1 : backupRoot + // Filter 2 : time range filter + // Filter 3 : table filter + BackupInfo.Filter destinationFilter = info -> info.getBackupRootDir().equals(backupDest); + + BackupInfo.Filter timeRangeFilter = info -> { + long time = info.getStartTs(); + return time >= startRangeTime && time <= endRangeTime; + }; + + BackupInfo.Filter tableFilter = info -> { + List tables = info.getTableNames(); + return !Collections.disjoint(allTables, tables); + }; + + BackupInfo.Filter typeFilter = info -> info.getType() == BackupType.INCREMENTAL; + BackupInfo.Filter stateFilter = info -> info.getState() == BackupState.COMPLETE; + + List allInfos = table.getBackupHistory(-1, destinationFilter, timeRangeFilter, + tableFilter, typeFilter, stateFilter); + if (allInfos.size() != allBackups.size()) { + // Yes we have at least one hole in backup image sequence + List missingIds = new ArrayList<>(); + for (BackupInfo info : allInfos) { + if (allBackups.contains(info.getBackupId())) { + continue; + } + missingIds.add(info.getBackupId()); + } + String errMsg = + "Sequence of backup ids has 'holes'. The following backup images must be added:" + + org.apache.hadoop.util.StringUtils.join(",", missingIds); + throw new IOException(errMsg); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java new file mode 100644 index 000000000000..ce9c5bbe8fae --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -0,0 +1,1123 @@ +/* + * 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 static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_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_KEEP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_LIST; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_BACKUP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_DESC; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; +import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; + +/** + * General backup commands, options and usage messages + */ +@InterfaceAudience.Private +public final class BackupCommands { + public final static String INCORRECT_USAGE = "Incorrect usage"; + + public final static String TOP_LEVEL_NOT_ALLOWED = + "Top level (root) folder is not allowed to be a backup destination"; + + public static final String USAGE = "Usage: hbase backup COMMAND [command-specific arguments]\n" + + "where COMMAND is one of:\n" + " create create a new backup image\n" + + " delete delete an existing backup image\n" + + " describe show the detailed information of a backup image\n" + + " history show history of all successful backups\n" + + " progress show the progress of the latest backup request\n" + + " set backup set management\n" + " repair repair backup system table\n" + + " merge merge backup images\n" + + "Run \'hbase backup COMMAND -h\' to see help message for each command\n"; + + public static final String CREATE_CMD_USAGE = + "Usage: hbase backup create [options]\n" + + " type \"full\" to create a full backup image\n" + + " \"incremental\" to create an incremental backup image\n" + + " backup_path Full path to store the backup image\n"; + + public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress \n" + + " backup_id Backup image id (optional). If no id specified, the command will show\n" + + " progress for currently running backup session."; + public static final String NO_INFO_FOUND = "No info was found for backup id: "; + public static final String NO_ACTIVE_SESSION_FOUND = "No active backup sessions found."; + + public static final String DESCRIBE_CMD_USAGE = + "Usage: hbase backup describe \n" + " backup_id Backup image id\n"; + + public static final String HISTORY_CMD_USAGE = "Usage: hbase backup history [options]"; + + public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete [options]"; + + public static final String REPAIR_CMD_USAGE = "Usage: hbase backup repair\n"; + + public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n" + + " name Backup set name\n" + " tables Comma separated list of tables.\n" + + "COMMAND is one of:\n" + " add add tables to a set, create a set if needed\n" + + " remove remove tables from a set\n" + + " list list all backup sets in the system\n" + " describe describe set\n" + + " delete delete backup set\n"; + public static final String MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n" + + " backup_ids Comma separated list of backup image ids.\n"; + + public static final String USAGE_FOOTER = ""; + + public static abstract class Command extends Configured { + CommandLine cmdline; + Connection conn; + + Command(Configuration conf) { + if (conf == null) { + conf = HBaseConfiguration.create(); + } + setConf(conf); + } + + public void execute() throws IOException { + if (cmdline.hasOption("h") || cmdline.hasOption("help")) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + // Create connection + conn = ConnectionFactory.createConnection(getConf()); + if (requiresNoActiveSession()) { + // Check active session + try (BackupSystemTable table = new BackupSystemTable(conn)) { + List sessions = table.getBackupInfos(BackupState.RUNNING); + + if (sessions.size() > 0) { + System.err.println("Found backup session in a RUNNING state: "); + System.err.println(sessions.get(0)); + System.err.println("This may indicate that a previous session has failed abnormally."); + System.err.println("In this case, backup recovery is recommended."); + throw new IOException("Active session found, aborted command execution"); + } + } + } + if (requiresConsistentState()) { + // Check failed delete + try (BackupSystemTable table = new BackupSystemTable(conn)) { + String[] ids = table.getListOfBackupIdsFromDeleteOperation(); + + if (ids != null && ids.length > 0) { + System.err.println("Found failed backup DELETE coommand. "); + System.err.println("Backup system recovery is required."); + throw new IOException("Failed backup DELETE found, aborted command execution"); + } + + ids = table.getListOfBackupIdsFromMergeOperation(); + if (ids != null && ids.length > 0) { + System.err.println("Found failed backup MERGE coommand. "); + System.err.println("Backup system recovery is required."); + throw new IOException("Failed backup MERGE found, aborted command execution"); + } + } + } + } + + public void finish() throws IOException { + if (conn != null) { + conn.close(); + } + } + + protected abstract void printUsage(); + + /** + * The command can't be run if active backup session is in progress + * @return true if no active sessions are in progress + */ + protected boolean requiresNoActiveSession() { + return false; + } + + /** + * Command requires consistent state of a backup system Backup system may become inconsistent + * because of an abnormal termination of a backup session or delete command + * @return true, if yes + */ + protected boolean requiresConsistentState() { + return false; + } + } + + private BackupCommands() { + throw new AssertionError("Instantiating utility class..."); + } + + public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) { + Command cmd; + switch (type) { + case CREATE: + cmd = new CreateCommand(conf, cmdline); + break; + case DESCRIBE: + cmd = new DescribeCommand(conf, cmdline); + break; + case PROGRESS: + cmd = new ProgressCommand(conf, cmdline); + break; + case DELETE: + cmd = new DeleteCommand(conf, cmdline); + break; + case HISTORY: + cmd = new HistoryCommand(conf, cmdline); + break; + case SET: + cmd = new BackupSetCommand(conf, cmdline); + break; + case REPAIR: + cmd = new RepairCommand(conf, cmdline); + break; + case MERGE: + cmd = new MergeCommand(conf, cmdline); + break; + case HELP: + default: + cmd = new HelpCommand(conf, cmdline); + break; + } + return cmd; + } + + static int numOfArgs(String[] args) { + if (args == null) { + return 0; + } + + return args.length; + } + + public static class CreateCommand extends Command { + CreateCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + protected boolean requiresNoActiveSession() { + return true; + } + + @Override + protected boolean requiresConsistentState() { + return true; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String[] args = cmdline.getArgs(); + if (args.length != 3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + if ( + !BackupType.FULL.toString().equalsIgnoreCase(args[1]) + && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1]) + ) { + System.out.println("ERROR: invalid backup type: " + args[1]); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + if (!verifyPath(args[2])) { + System.out.println("ERROR: invalid backup destination: " + args[2]); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String targetBackupDir = args[2]; + // Check if backup destination is top level (root) folder - not allowed + if (isRootFolder(targetBackupDir)) { + throw new IOException(TOP_LEVEL_NOT_ALLOWED); + } + String tables; + + // Check if we have both: backup set and list of tables + if (cmdline.hasOption(OPTION_TABLE) && cmdline.hasOption(OPTION_SET)) { + System.out + .println("ERROR: You can specify either backup set or list" + " of tables, but not both"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + // Creates connection + super.execute(); + // Check backup set + String setName = null; + if (cmdline.hasOption(OPTION_SET)) { + setName = cmdline.getOptionValue(OPTION_SET); + tables = getTablesForSet(setName); + + if (tables == null) { + System.out + .println("ERROR: Backup set '" + setName + "' is either empty or does not exist"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } else { + tables = cmdline.getOptionValue(OPTION_TABLE); + } + int bandwidth = cmdline.hasOption(OPTION_BANDWIDTH) + ? Integer.parseInt(cmdline.getOptionValue(OPTION_BANDWIDTH)) + : -1; + int workers = cmdline.hasOption(OPTION_WORKERS) + ? Integer.parseInt(cmdline.getOptionValue(OPTION_WORKERS)) + : -1; + + if (cmdline.hasOption(OPTION_YARN_QUEUE_NAME)) { + String queueName = cmdline.getOptionValue(OPTION_YARN_QUEUE_NAME); + // Set system property value for MR job + System.setProperty("mapreduce.job.queuename", queueName); + } + + try (BackupAdminImpl admin = new BackupAdminImpl(conn)) { + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase())) + .withTableList( + tables != null ? Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null) + .withTargetRootDir(targetBackupDir).withTotalTasks(workers) + .withBandwidthPerTasks(bandwidth).withBackupSetName(setName).build(); + String backupId = admin.backupTables(request); + System.out.println("Backup session " + backupId + " finished. Status: SUCCESS"); + } catch (IOException e) { + System.out.println("Backup session finished. Status: FAILURE"); + throw e; + } + } + + private boolean isRootFolder(String targetBackupDir) { + Path p = new Path(targetBackupDir); + return p.isRoot(); + } + + private boolean verifyPath(String path) { + try { + Path p = new Path(path); + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + URI uri = p.toUri(); + + if (uri.getScheme() == null) { + return false; + } + + FileSystem.get(uri, conf); + return true; + } catch (Exception e) { + return false; + } + } + + private String getTablesForSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List tables = table.describeBackupSet(name); + + if (tables == null) { + return null; + } + + return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } + + @Override + protected void printUsage() { + System.out.println(CREATE_CMD_USAGE); + Options options = new Options(); + options.addOption(OPTION_WORKERS, true, OPTION_WORKERS_DESC); + options.addOption(OPTION_BANDWIDTH, true, OPTION_BANDWIDTH_DESC); + options.addOption(OPTION_SET, true, OPTION_SET_BACKUP_DESC); + options.addOption(OPTION_TABLE, true, OPTION_TABLE_LIST_DESC); + options.addOption(OPTION_YARN_QUEUE_NAME, true, OPTION_YARN_QUEUE_NAME_DESC); + options.addOption(OPTION_DEBUG, false, OPTION_DEBUG_DESC); + + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + } + } + + public static class HelpCommand extends Command { + HelpCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] args = cmdline.getArgs(); + if (args == null || args.length == 0) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + if (args.length != 2) { + System.out.println("ERROR: Only supports help message of a single command type"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String type = args[1]; + + if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) { + System.out.println(CREATE_CMD_USAGE); + } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) { + System.out.println(DESCRIBE_CMD_USAGE); + } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) { + System.out.println(HISTORY_CMD_USAGE); + } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) { + System.out.println(PROGRESS_CMD_USAGE); + } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) { + System.out.println(DELETE_CMD_USAGE); + } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) { + System.out.println(SET_CMD_USAGE); + } else { + System.out.println("Unknown command : " + type); + printUsage(); + } + } + + @Override + protected void printUsage() { + System.out.println(USAGE); + } + } + + public static class DescribeCommand extends Command { + DescribeCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String[] args = cmdline.getArgs(); + if (args.length != 2) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + super.execute(); + + String backupId = args[1]; + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + BackupInfo info = sysTable.readBackupInfo(backupId); + if (info == null) { + System.out.println("ERROR: " + backupId + " does not exist"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + System.out.println(info.getShortDescription()); + } + } + + @Override + protected void printUsage() { + System.out.println(DESCRIBE_CMD_USAGE); + } + } + + public static class ProgressCommand extends Command { + ProgressCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 1) { + System.out.println( + "No backup id was specified, " + "will retrieve the most recent (ongoing) session"); + } + String[] args = cmdline == null ? null : cmdline.getArgs(); + if (args != null && args.length > 2) { + System.err.println("ERROR: wrong number of arguments: " + args.length); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + super.execute(); + + String backupId = (args == null || args.length <= 1) ? null : args[1]; + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + BackupInfo info = null; + + if (backupId != null) { + info = sysTable.readBackupInfo(backupId); + } else { + List infos = sysTable.getBackupInfos(BackupState.RUNNING); + if (infos != null && infos.size() > 0) { + info = infos.get(0); + backupId = info.getBackupId(); + System.out.println("Found ongoing session with backupId=" + backupId); + } + } + int progress = info == null ? -1 : info.getProgress(); + if (progress < 0) { + if (backupId != null) { + System.out.println(NO_INFO_FOUND + backupId); + } else { + System.err.println(NO_ACTIVE_SESSION_FOUND); + } + } else { + System.out.println(backupId + " progress=" + progress + "%"); + } + } + } + + @Override + protected void printUsage() { + System.out.println(PROGRESS_CMD_USAGE); + } + } + + public static class DeleteCommand extends Command { + DeleteCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + protected boolean requiresNoActiveSession() { + return true; + } + + @Override + public void execute() throws IOException { + + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 1) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + if (!cmdline.hasOption(OPTION_KEEP) && !cmdline.hasOption(OPTION_LIST)) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + super.execute(); + if (cmdline.hasOption(OPTION_KEEP)) { + executeDeleteOlderThan(cmdline); + } else if (cmdline.hasOption(OPTION_LIST)) { + executeDeleteListOfBackups(cmdline); + } + } + + private void executeDeleteOlderThan(CommandLine cmdline) throws IOException { + String value = cmdline.getOptionValue(OPTION_KEEP); + int days = 0; + try { + days = Integer.parseInt(value); + } catch (NumberFormatException e) { + throw new IOException(value + " is not an integer number"); + } + final long fdays = days; + BackupInfo.Filter dateFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + long currentTime = EnvironmentEdgeManager.currentTime(); + long maxTsToDelete = currentTime - fdays * 24 * 3600 * 1000; + return info.getCompleteTs() <= maxTsToDelete; + } + }; + List history = null; + try (final BackupSystemTable sysTable = new BackupSystemTable(conn); + BackupAdminImpl admin = new BackupAdminImpl(conn)) { + history = sysTable.getBackupHistory(-1, dateFilter); + String[] backupIds = convertToBackupIds(history); + int deleted = admin.deleteBackups(backupIds); + System.out.println("Deleted " + deleted + " backups. Total older than " + days + " days: " + + backupIds.length); + } catch (IOException e) { + System.err.println("Delete command FAILED. Please run backup repair tool to restore backup " + + "system integrity"); + throw e; + } + } + + private String[] convertToBackupIds(List history) { + String[] ids = new String[history.size()]; + for (int i = 0; i < ids.length; i++) { + ids[i] = history.get(i).getBackupId(); + } + return ids; + } + + private void executeDeleteListOfBackups(CommandLine cmdline) throws IOException { + String value = cmdline.getOptionValue(OPTION_LIST); + String[] backupIds = value.split(","); + + try (BackupAdminImpl admin = new BackupAdminImpl(conn)) { + int deleted = admin.deleteBackups(backupIds); + System.out.println("Deleted " + deleted + " backups. Total requested: " + backupIds.length); + } catch (IOException e) { + System.err.println("Delete command FAILED. Please run backup repair tool to restore backup " + + "system integrity"); + throw e; + } + + } + + @Override + protected void printUsage() { + System.out.println(DELETE_CMD_USAGE); + Options options = new Options(); + options.addOption(OPTION_KEEP, true, OPTION_KEEP_DESC); + options.addOption(OPTION_LIST, true, OPTION_BACKUP_LIST_DESC); + + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + + } + } + + public static class RepairCommand extends Command { + RepairCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + + String[] args = cmdline == null ? null : cmdline.getArgs(); + if (args != null && args.length > 1) { + System.err.println("ERROR: wrong number of arguments: " + args.length); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + // Failed backup + BackupInfo backupInfo; + List list = sysTable.getBackupInfos(BackupState.RUNNING); + if (list.size() == 0) { + // No failed sessions found + System.out.println("REPAIR status: no failed sessions found." + + " Checking failed delete backup operation ..."); + repairFailedBackupDeletionIfAny(conn, sysTable); + repairFailedBackupMergeIfAny(conn, sysTable); + return; + } + backupInfo = list.get(0); + // If this is a cancel exception, then we've already cleaned. + // set the failure timestamp of the overall backup + backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); + // set failure message + backupInfo.setFailedMsg("REPAIR status: repaired after failure:\n" + backupInfo); + // set overall backup status: failed + backupInfo.setState(BackupState.FAILED); + // compose the backup failed data + String backupFailedData = "BackupId=" + backupInfo.getBackupId() + ",startts=" + + backupInfo.getStartTs() + ",failedts=" + backupInfo.getCompleteTs() + ",failedphase=" + + backupInfo.getPhase() + ",failedmessage=" + backupInfo.getFailedMsg(); + System.out.println(backupFailedData); + TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf); + // If backup session is updated to FAILED state - means we + // processed recovery already. + sysTable.updateBackupInfo(backupInfo); + sysTable.finishBackupExclusiveOperation(); + System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo); + } + } + + private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable) + throws IOException { + String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation(); + if (backupIds == null || backupIds.length == 0) { + System.out.println("No failed backup DELETE operation found"); + // Delete backup table snapshot if exists + BackupSystemTable.deleteSnapshot(conn); + return; + } + System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds)); + System.out.println("Running DELETE again ..."); + // Restore table from snapshot + BackupSystemTable.restoreFromSnapshot(conn); + // Finish previous failed session + sysTable.finishBackupExclusiveOperation(); + try (BackupAdmin admin = new BackupAdminImpl(conn)) { + admin.deleteBackups(backupIds); + } + System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds)); + } + + public static void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable) + throws IOException { + + String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation(); + if (backupIds == null || backupIds.length == 0) { + System.out.println("No failed backup MERGE operation found"); + // Delete backup table snapshot if exists + BackupSystemTable.deleteSnapshot(conn); + return; + } + System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds)); + // Check if backup .tmp exists + BackupInfo bInfo = sysTable.readBackupInfo(backupIds[0]); + String backupRoot = bInfo.getBackupRootDir(); + FileSystem fs = FileSystem.get(new Path(backupRoot).toUri(), new Configuration()); + String backupId = BackupUtils.findMostRecentBackupId(backupIds); + Path tmpPath = HBackupFileSystem.getBackupTmpDirPathForBackupId(backupRoot, backupId); + if (fs.exists(tmpPath)) { + // Move data back + Path destPath = HBackupFileSystem.getBackupPath(backupRoot, backupId); + if (!fs.delete(destPath, true)) { + System.out.println("Failed to delete " + destPath); + } + boolean res = fs.rename(tmpPath, destPath); + if (!res) { + throw new IOException( + "MERGE repair: failed to rename from " + tmpPath + " to " + destPath); + } + System.out + .println("MERGE repair: renamed from " + tmpPath + " to " + destPath + " res=" + res); + } else { + checkRemoveBackupImages(fs, backupRoot, backupIds); + } + // Restore table from snapshot + BackupSystemTable.restoreFromSnapshot(conn); + // Unlock backup system + sysTable.finishBackupExclusiveOperation(); + // Finish previous failed session + sysTable.finishMergeOperation(); + + System.out.println("MERGE repair operation finished OK: " + StringUtils.join(backupIds)); + } + + private static void checkRemoveBackupImages(FileSystem fs, String backupRoot, + String[] backupIds) throws IOException { + String mergedBackupId = BackupUtils.findMostRecentBackupId(backupIds); + for (String backupId : backupIds) { + if (backupId.equals(mergedBackupId)) { + continue; + } + Path path = HBackupFileSystem.getBackupPath(backupRoot, backupId); + if (fs.exists(path)) { + if (!fs.delete(path, true)) { + System.out.println("MERGE repair removing: " + path + " - FAILED"); + } else { + System.out.println("MERGE repair removing: " + path + " - OK"); + } + } + } + } + + @Override + protected void printUsage() { + System.out.println(REPAIR_CMD_USAGE); + } + } + + public static class MergeCommand extends Command { + MergeCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + protected boolean requiresNoActiveSession() { + return true; + } + + @Override + protected boolean requiresConsistentState() { + return true; + } + + @Override + public void execute() throws IOException { + super.execute(); + + String[] args = cmdline == null ? null : cmdline.getArgs(); + if (args == null || (args.length != 2)) { + System.err + .println("ERROR: wrong number of arguments: " + (args == null ? null : args.length)); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] backupIds = args[1].split(","); + if (backupIds.length < 2) { + String msg = "ERROR: can not merge a single backup image. " + + "Number of images must be greater than 1."; + System.err.println(msg); + throw new IOException(msg); + + } + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupAdminImpl admin = new BackupAdminImpl(conn)) { + admin.mergeBackups(backupIds); + } + } + + @Override + protected void printUsage() { + System.out.println(MERGE_CMD_USAGE); + } + } + + public static class HistoryCommand extends Command { + private final static int DEFAULT_HISTORY_LENGTH = 10; + + HistoryCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + int n = parseHistoryLength(); + final TableName tableName = getTableName(); + final String setName = getTableSetName(); + BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + if (tableName == null) { + return true; + } + + List names = info.getTableNames(); + return names.contains(tableName); + } + }; + BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + if (setName == null) { + return true; + } + + String backupId = info.getBackupId(); + return backupId.startsWith(setName); + } + }; + Path backupRootPath = getBackupRootPath(); + List history; + if (backupRootPath == null) { + // Load from backup system table + super.execute(); + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter); + } + } else { + // load from backup FS + history = + BackupUtils.getHistory(getConf(), n, backupRootPath, tableNameFilter, tableSetFilter); + } + for (BackupInfo info : history) { + System.out.println(info.getShortDescription()); + } + } + + private Path getBackupRootPath() throws IOException { + String value = null; + try { + value = cmdline.getOptionValue(OPTION_PATH); + + if (value == null) { + return null; + } + + return new Path(value); + } catch (IllegalArgumentException e) { + System.out.println("ERROR: Illegal argument for backup root path: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + private TableName getTableName() throws IOException { + String value = cmdline.getOptionValue(OPTION_TABLE); + + if (value == null) { + return null; + } + + try { + return TableName.valueOf(value); + } catch (IllegalArgumentException e) { + System.out.println("Illegal argument for table name: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + private String getTableSetName() { + return cmdline.getOptionValue(OPTION_SET); + } + + private int parseHistoryLength() throws IOException { + String value = cmdline.getOptionValue(OPTION_RECORD_NUMBER); + try { + if (value == null) { + return DEFAULT_HISTORY_LENGTH; + } + + return Integer.parseInt(value); + } catch (NumberFormatException e) { + System.out.println("Illegal argument for history length: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + @Override + protected void printUsage() { + System.out.println(HISTORY_CMD_USAGE); + Options options = new Options(); + options.addOption(OPTION_RECORD_NUMBER, true, OPTION_RECORD_NUMBER_DESC); + options.addOption(OPTION_PATH, true, OPTION_PATH_DESC); + options.addOption(OPTION_TABLE, true, OPTION_TABLE_DESC); + options.addOption(OPTION_SET, true, OPTION_SET_DESC); + + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + } + } + + public static class BackupSetCommand extends Command { + private final static String SET_ADD_CMD = "add"; + private final static String SET_REMOVE_CMD = "remove"; + private final static String SET_DELETE_CMD = "delete"; + private final static String SET_DESCRIBE_CMD = "describe"; + private final static String SET_LIST_CMD = "list"; + + BackupSetCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + // Command-line must have at least one element + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] args = cmdline.getArgs(); + String cmdStr = args[1]; + BackupCommand cmd = getCommand(cmdStr); + + switch (cmd) { + case SET_ADD: + processSetAdd(args); + break; + case SET_REMOVE: + processSetRemove(args); + break; + case SET_DELETE: + processSetDelete(args); + break; + case SET_DESCRIBE: + processSetDescribe(args); + break; + case SET_LIST: + processSetList(); + break; + default: + break; + } + } + + private void processSetList() throws IOException { + super.execute(); + + // List all backup set names + // does not expect any args + try (BackupAdminImpl admin = new BackupAdminImpl(conn)) { + List list = admin.listBackupSets(); + for (BackupSet bs : list) { + System.out.println(bs); + } + } + } + + private void processSetDescribe(String[] args) throws IOException { + if (args == null || args.length != 3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + super.execute(); + + String setName = args[2]; + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + List tables = sysTable.describeBackupSet(setName); + BackupSet set = tables == null ? null : new BackupSet(setName, tables); + if (set == null) { + System.out.println("Set '" + setName + "' does not exist."); + } else { + System.out.println(set); + } + } + } + + private void processSetDelete(String[] args) throws IOException { + if (args == null || args.length != 3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + super.execute(); + + String setName = args[2]; + try (final BackupAdminImpl admin = new BackupAdminImpl(conn)) { + boolean result = admin.deleteBackupSet(setName); + if (result) { + System.out.println("Delete set " + setName + " OK."); + } else { + System.out.println("Set " + setName + " does not exist"); + } + } + } + + private void processSetRemove(String[] args) throws IOException { + if (args == null || args.length != 4) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + super.execute(); + + String setName = args[2]; + String[] tables = args[3].split(","); + TableName[] tableNames = toTableNames(tables); + try (final BackupAdminImpl admin = new BackupAdminImpl(conn)) { + admin.removeFromBackupSet(setName, tableNames); + } + } + + private TableName[] toTableNames(String[] tables) { + TableName[] arr = new TableName[tables.length]; + for (int i = 0; i < tables.length; i++) { + arr[i] = TableName.valueOf(tables[i]); + } + return arr; + } + + private void processSetAdd(String[] args) throws IOException { + if (args == null || args.length != 4) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + super.execute(); + String setName = args[2]; + TableName[] tableNames = + Splitter.on(',').splitToStream(args[3]).map(TableName::valueOf).toArray(TableName[]::new); + try (final BackupAdminImpl admin = new BackupAdminImpl(conn)) { + admin.addToBackupSet(setName, tableNames); + } + } + + private BackupCommand getCommand(String cmdStr) throws IOException { + switch (cmdStr) { + case SET_ADD_CMD: + return BackupCommand.SET_ADD; + case SET_REMOVE_CMD: + return BackupCommand.SET_REMOVE; + case SET_DELETE_CMD: + return BackupCommand.SET_DELETE; + case SET_DESCRIBE_CMD: + return BackupCommand.SET_DESCRIBE; + case SET_LIST_CMD: + return BackupCommand.SET_LIST; + default: + System.out.println("ERROR: Unknown command for 'set' :" + cmdStr); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + @Override + protected void printUsage() { + System.out.println(SET_CMD_USAGE); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java new file mode 100644 index 000000000000..8dd262cbb88f --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Backup exception + */ +@SuppressWarnings("serial") +@InterfaceAudience.Private +public class BackupException extends HBaseIOException { + private BackupInfo info; + + /** + * Some exception happened for a backup and don't even know the backup that it was about + * @param msg Full description of the failure + */ + public BackupException(String msg) { + super(msg); + } + + /** + * Some exception happened for a backup with a cause + * @param cause the cause + */ + public BackupException(Throwable cause) { + super(cause); + } + + /** + * Exception for the given backup that has no previous root cause + * @param msg reason why the backup failed + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, BackupInfo desc) { + super(msg); + this.info = desc; + } + + /** + * Exception for the given backup due to another exception + * @param msg reason why the backup failed + * @param cause root cause of the failure + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, Throwable cause, BackupInfo desc) { + super(msg, cause); + this.info = desc; + } + + /** + * Exception when the description of the backup cannot be determined, due to some other root cause + * @param message description of what caused the failure + * @param e root cause + */ + public BackupException(String message, Exception e) { + super(message, e); + } + + public BackupInfo getBackupInfo() { + return this.info; + } + +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java new file mode 100644 index 000000000000..a543b577b7ae --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -0,0 +1,521 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupHFileCleaner; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupObserver; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.master.BackupLogCleaner; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; +import org.apache.hadoop.hbase.procedure.ProcedureManagerHost; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Handles backup requests, creates backup info records in backup system table to keep track of + * backup sessions, dispatches backup request. + */ +@InterfaceAudience.Private +public class BackupManager implements Closeable { + // in seconds + public final static String BACKUP_EXCLUSIVE_OPERATION_TIMEOUT_SECONDS_KEY = + "hbase.backup.exclusive.op.timeout.seconds"; + // In seconds + private final static int DEFAULT_BACKUP_EXCLUSIVE_OPERATION_TIMEOUT = 3600; + private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class); + + protected Configuration conf = null; + protected BackupInfo backupInfo = null; + protected BackupSystemTable systemTable; + protected final Connection conn; + + /** + * Backup manager constructor. + * @param conn connection + * @param conf configuration + * @throws IOException exception + */ + public BackupManager(Connection conn, Configuration conf) throws IOException { + if ( + !conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.BACKUP_ENABLE_DEFAULT) + ) { + throw new BackupException("HBase backup is not enabled. Check your " + + BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting."); + } + this.conf = conf; + this.conn = conn; + this.systemTable = new BackupSystemTable(conn); + } + + /** + * Returns backup info + */ + protected BackupInfo getBackupInfo() { + return backupInfo; + } + + /** + * This method modifies the master's configuration in order to inject backup-related features + * (TESTs only) + * @param conf configuration + */ + public static void decorateMasterConfiguration(Configuration conf) { + if (!isBackupEnabled(conf)) { + return; + } + // Add WAL archive cleaner plug-in + String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS); + String cleanerClass = BackupLogCleaner.class.getCanonicalName(); + if (!plugins.contains(cleanerClass)) { + conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass); + } + + String classes = conf.get(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY); + String masterProcedureClass = LogRollMasterProcedureManager.class.getName(); + if (classes == null) { + conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, masterProcedureClass); + } else if (!classes.contains(masterProcedureClass)) { + conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, + classes + "," + masterProcedureClass); + } + + plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); + conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, + (plugins == null ? "" : plugins + ",") + BackupHFileCleaner.class.getName()); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Added log cleaner: {}. Added master procedure manager: {}." + + "Added master procedure manager: {}", + cleanerClass, masterProcedureClass, BackupHFileCleaner.class.getName()); + } + } + + /** + * This method modifies the Region Server configuration in order to inject backup-related features + * TESTs only. + * @param conf configuration + */ + public static void decorateRegionServerConfiguration(Configuration conf) { + if (!isBackupEnabled(conf)) { + return; + } + + String classes = conf.get(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY); + String regionProcedureClass = LogRollRegionServerProcedureManager.class.getName(); + if (classes == null) { + conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, regionProcedureClass); + } else if (!classes.contains(regionProcedureClass)) { + conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, + classes + "," + regionProcedureClass); + } + String coproc = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY); + String regionObserverClass = BackupObserver.class.getName(); + conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + (coproc == null ? "" : coproc + ",") + regionObserverClass); + if (LOG.isDebugEnabled()) { + LOG.debug("Added region procedure manager: {}. Added region observer: {}", + regionProcedureClass, regionObserverClass); + } + } + + public static boolean isBackupEnabled(Configuration conf) { + return conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.BACKUP_ENABLE_DEFAULT); + } + + /** + * Get configuration n + */ + Configuration getConf() { + return conf; + } + + /** + * Stop all the work of backup. + */ + @Override + public void close() { + if (systemTable != null) { + try { + systemTable.close(); + } catch (Exception e) { + LOG.error(e.toString(), e); + } + } + } + + /** + * Creates a backup info based on input backup request. + * @param backupId backup id + * @param type type + * @param tableList table list + * @param targetRootDir root dir + * @param workers number of parallel workers + * @param bandwidth bandwidth per worker in MB per sec n * @throws BackupException exception + */ + public BackupInfo createBackupInfo(String backupId, BackupType type, List tableList, + String targetRootDir, int workers, long bandwidth) throws BackupException { + if (targetRootDir == null) { + throw new BackupException("Wrong backup request parameter: target backup root directory"); + } + + if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) { + // If table list is null for full backup, which means backup all tables. Then fill the table + // list with all user tables from meta. It no table available, throw the request exception. + List htds = null; + try (Admin admin = conn.getAdmin()) { + htds = admin.listTableDescriptors(); + } catch (Exception e) { + throw new BackupException(e); + } + + if (htds == null) { + throw new BackupException("No table exists for full backup of all tables."); + } else { + tableList = new ArrayList<>(); + for (TableDescriptor hTableDescriptor : htds) { + TableName tn = hTableDescriptor.getTableName(); + if (tn.equals(BackupSystemTable.getTableName(conf))) { + // skip backup system table + continue; + } + tableList.add(hTableDescriptor.getTableName()); + } + + LOG.info("Full backup all the tables available in the cluster: {}", tableList); + } + } + + // there are one or more tables in the table list + backupInfo = new BackupInfo(backupId, type, tableList.toArray(new TableName[tableList.size()]), + targetRootDir); + backupInfo.setBandwidth(bandwidth); + backupInfo.setWorkers(workers); + return backupInfo; + } + + /** + * Check if any ongoing backup. Currently, we only reply on checking status in backup system + * table. We need to consider to handle the case of orphan records in the future. Otherwise, all + * the coming request will fail. + * @return the ongoing backup id if on going backup exists, otherwise null + * @throws IOException exception + */ + private String getOngoingBackupId() throws IOException { + ArrayList sessions = systemTable.getBackupInfos(BackupState.RUNNING); + if (sessions.size() == 0) { + return null; + } + return sessions.get(0).getBackupId(); + } + + /** + * Start the backup manager service. + * @throws IOException exception + */ + public void initialize() throws IOException { + String ongoingBackupId = this.getOngoingBackupId(); + if (ongoingBackupId != null) { + LOG.info("There is a ongoing backup {}" + + ". Can not launch new backup until no ongoing backup remains.", ongoingBackupId); + throw new BackupException("There is ongoing backup seesion."); + } + } + + public void setBackupInfo(BackupInfo backupInfo) { + this.backupInfo = backupInfo; + } + + /** + * Get direct ancestors of the current backup. + * @param backupInfo The backup info for the current backup + * @return The ancestors for the current backup + * @throws IOException exception + */ + public ArrayList getAncestors(BackupInfo backupInfo) throws IOException { + LOG.debug("Getting the direct ancestors of the current backup {}", backupInfo.getBackupId()); + + ArrayList ancestors = new ArrayList<>(); + + // full backup does not have ancestor + if (backupInfo.getType() == BackupType.FULL) { + LOG.debug("Current backup is a full backup, no direct ancestor for it."); + return ancestors; + } + + // get all backup history list in descending order + ArrayList allHistoryList = getBackupHistory(true); + for (BackupInfo backup : allHistoryList) { + + BackupImage.Builder builder = BackupImage.newBuilder(); + + BackupImage image = builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames()) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + + // Only direct ancestors for a backup are required and not entire history of backup for this + // table resulting in verifying all of the previous backups which is unnecessary and backup + // paths need not be valid beyond the lifetime of a backup. + // + // RootDir is way of grouping a single backup including one full and many incremental backups + if (!image.getRootDir().equals(backupInfo.getBackupRootDir())) { + continue; + } + + // add the full backup image as an ancestor until the last incremental backup + if (backup.getType().equals(BackupType.FULL)) { + // check the backup image coverage, if previous image could be covered by the newer ones, + // then no need to add + if (!BackupManifest.canCoverImage(ancestors, image)) { + ancestors.add(image); + } + } else { + // found last incremental backup, if previously added full backup ancestor images can cover + // it, then this incremental ancestor is not the dependent of the current incremental + // backup, that is to say, this is the backup scope boundary of current table set. + // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing + // incremental backup + if (BackupManifest.canCoverImage(ancestors, image)) { + LOG.debug("Met the backup boundary of the current table set:"); + for (BackupImage image1 : ancestors) { + LOG.debug(" BackupID={}, BackupDir={}", image1.getBackupId(), image1.getRootDir()); + } + } else { + Path logBackupPath = + HBackupFileSystem.getBackupPath(backup.getBackupRootDir(), backup.getBackupId()); + LOG.debug( + "Current backup has an incremental backup ancestor, " + + "touching its image manifest in {}" + " to construct the dependency.", + logBackupPath.toString()); + BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath); + BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage(); + ancestors.add(lastIncrImage); + + LOG.debug("Last dependent incremental backup image: {BackupID={}" + "BackupDir={}}", + lastIncrImage.getBackupId(), lastIncrImage.getRootDir()); + } + } + } + LOG.debug("Got {} ancestors for the current backup.", ancestors.size()); + return ancestors; + } + + /** + * Get the direct ancestors of this backup for one table involved. + * @param backupInfo backup info + * @param table table + * @return backupImages on the dependency list + * @throws IOException exception + */ + public ArrayList getAncestors(BackupInfo backupInfo, TableName table) + throws IOException { + ArrayList ancestors = getAncestors(backupInfo); + ArrayList tableAncestors = new ArrayList<>(); + for (BackupImage image : ancestors) { + if (image.hasTable(table)) { + tableAncestors.add(image); + if (image.getType() == BackupType.FULL) { + break; + } + } + } + return tableAncestors; + } + + /* + * backup system table operations + */ + + /** + * Updates status (state) of a backup session in a persistent store + * @param context context + * @throws IOException exception + */ + public void updateBackupInfo(BackupInfo context) throws IOException { + systemTable.updateBackupInfo(context); + } + + /** + * Starts new backup session + * @throws IOException if active session already exists + */ + public void startBackupSession() throws IOException { + long startTime = EnvironmentEdgeManager.currentTime(); + long timeout = conf.getInt(BACKUP_EXCLUSIVE_OPERATION_TIMEOUT_SECONDS_KEY, + DEFAULT_BACKUP_EXCLUSIVE_OPERATION_TIMEOUT) * 1000L; + long lastWarningOutputTime = 0; + while (EnvironmentEdgeManager.currentTime() - startTime < timeout) { + try { + systemTable.startBackupExclusiveOperation(); + return; + } catch (IOException e) { + if (e instanceof ExclusiveOperationException) { + // sleep, then repeat + try { + Thread.sleep(1000); + } catch (InterruptedException e1) { + // Restore the interrupted status + Thread.currentThread().interrupt(); + } + if ( + lastWarningOutputTime == 0 + || (EnvironmentEdgeManager.currentTime() - lastWarningOutputTime) > 60000 + ) { + lastWarningOutputTime = EnvironmentEdgeManager.currentTime(); + LOG.warn("Waiting to acquire backup exclusive lock for {}s", + +(lastWarningOutputTime - startTime) / 1000); + } + } else { + throw e; + } + } + } + throw new IOException( + "Failed to acquire backup system table exclusive lock after " + timeout / 1000 + "s"); + } + + /** + * Finishes active backup session + * @throws IOException if no active session + */ + public void finishBackupSession() throws IOException { + systemTable.finishBackupExclusiveOperation(); + } + + /** + * Read the last backup start code (timestamp) of last successful backup. Will return null if + * there is no startcode stored in backup system table or the value is of length 0. These two + * cases indicate there is no successful backup completed so far. + * @return the timestamp of a last successful backup + * @throws IOException exception + */ + public String readBackupStartCode() throws IOException { + return systemTable.readBackupStartCode(backupInfo.getBackupRootDir()); + } + + /** + * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte. + * @param startCode start code + * @throws IOException exception + */ + public void writeBackupStartCode(Long startCode) throws IOException { + systemTable.writeBackupStartCode(startCode, backupInfo.getBackupRootDir()); + } + + /** + * Get the RS log information after the last log roll from backup system table. + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult() throws IOException { + return systemTable.readRegionServerLastLogRollResult(backupInfo.getBackupRootDir()); + } + + public Pair>>>>, List> + readBulkloadRows(List tableList) throws IOException { + return systemTable.readBulkloadRows(tableList); + } + + public void deleteBulkLoadedRows(List rows) throws IOException { + systemTable.deleteBulkLoadedRows(rows); + } + + /** + * Get all completed backup information (in desc order by time) + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public List getBackupHistory() throws IOException { + return systemTable.getBackupHistory(); + } + + public ArrayList getBackupHistory(boolean completed) throws IOException { + return systemTable.getBackupHistory(completed); + } + + /** + * Write the current timestamps for each regionserver to backup system table after a successful + * full or incremental backup. Each table may have a different set of log timestamps. The saved + * timestamp is of the last log file that was backed up already. + * @param tables tables + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, Map newTimestamps) + throws IOException { + systemTable.writeRegionServerLogTimestamp(tables, newTimestamps, backupInfo.getBackupRootDir()); + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public Map> readLogTimestampMap() throws IOException { + return systemTable.readLogTimestampMap(backupInfo.getBackupRootDir()); + } + + /** + * Return the current tables covered by incremental backup. + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet() throws IOException { + return systemTable.getIncrementalBackupTableSet(backupInfo.getBackupRootDir()); + } + + /** + * Adds set of tables to overall incremental backup table set + * @param tables tables + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables) throws IOException { + systemTable.addIncrementalBackupTableSet(tables, backupInfo.getBackupRootDir()); + } + + public Connection getConnection() { + return conn; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java new file mode 100644 index 000000000000..3a1cbd55c58e --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -0,0 +1,664 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.HBaseProtos; + +/** + * Backup manifest contains all the meta data of a backup image. The manifest info will be bundled + * as manifest file together with data. So that each backup image will contain all the info needed + * for restore. BackupManifest is a storage container for BackupImage. It is responsible for + * storing/reading backup image data and has some additional utility methods. + */ +@InterfaceAudience.Private +public class BackupManifest { + private static final Logger LOG = LoggerFactory.getLogger(BackupManifest.class); + + // manifest file name + public static final String MANIFEST_FILE_NAME = ".backup.manifest"; + + /** + * Backup image, the dependency graph is made up by series of backup images BackupImage contains + * all the relevant information to restore the backup and is used during restore operation + */ + public static class BackupImage implements Comparable { + static class Builder { + BackupImage image; + + Builder() { + image = new BackupImage(); + } + + Builder withBackupId(String backupId) { + image.setBackupId(backupId); + return this; + } + + Builder withType(BackupType type) { + image.setType(type); + return this; + } + + Builder withRootDir(String rootDir) { + image.setRootDir(rootDir); + return this; + } + + Builder withTableList(List tableList) { + image.setTableList(tableList); + return this; + } + + Builder withStartTime(long startTime) { + image.setStartTs(startTime); + return this; + } + + Builder withCompleteTime(long completeTime) { + image.setCompleteTs(completeTime); + return this; + } + + BackupImage build() { + return image; + } + + } + + private String backupId; + private BackupType type; + private String rootDir; + private List tableList; + private long startTs; + private long completeTs; + private ArrayList ancestors; + private Map> incrTimeRanges; + + static Builder newBuilder() { + return new Builder(); + } + + public BackupImage() { + super(); + } + + private BackupImage(String backupId, BackupType type, String rootDir, List tableList, + long startTs, long completeTs) { + this.backupId = backupId; + this.type = type; + this.rootDir = rootDir; + this.tableList = tableList; + this.startTs = startTs; + this.completeTs = completeTs; + } + + static BackupImage fromProto(BackupProtos.BackupImage im) { + String backupId = im.getBackupId(); + String rootDir = im.getBackupRootDir(); + long startTs = im.getStartTs(); + long completeTs = im.getCompleteTs(); + List tableListList = im.getTableListList(); + List tableList = new ArrayList<>(); + for (HBaseProtos.TableName tn : tableListList) { + tableList.add(ProtobufUtil.toTableName(tn)); + } + + List ancestorList = im.getAncestorsList(); + + BackupType type = im.getBackupType() == BackupProtos.BackupType.FULL + ? BackupType.FULL + : BackupType.INCREMENTAL; + + BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); + for (BackupProtos.BackupImage img : ancestorList) { + image.addAncestor(fromProto(img)); + } + image.setIncrTimeRanges(loadIncrementalTimestampMap(im)); + return image; + } + + BackupProtos.BackupImage toProto() { + BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder(); + builder.setBackupId(backupId); + builder.setCompleteTs(completeTs); + builder.setStartTs(startTs); + builder.setBackupRootDir(rootDir); + if (type == BackupType.FULL) { + builder.setBackupType(BackupProtos.BackupType.FULL); + } else { + builder.setBackupType(BackupProtos.BackupType.INCREMENTAL); + } + + for (TableName name : tableList) { + builder.addTableList(ProtobufUtil.toProtoTableName(name)); + } + + if (ancestors != null) { + for (BackupImage im : ancestors) { + builder.addAncestors(im.toProto()); + } + } + + setIncrementalTimestampMap(builder); + return builder.build(); + } + + private static Map> + loadIncrementalTimestampMap(BackupProtos.BackupImage proto) { + List list = proto.getTstMapList(); + + Map> incrTimeRanges = new HashMap<>(); + + if (list == null || list.size() == 0) { + return incrTimeRanges; + } + + for (BackupProtos.TableServerTimestamp tst : list) { + TableName tn = ProtobufUtil.toTableName(tst.getTableName()); + Map map = incrTimeRanges.get(tn); + if (map == null) { + map = new HashMap<>(); + incrTimeRanges.put(tn, map); + } + List listSt = tst.getServerTimestampList(); + for (BackupProtos.ServerTimestamp stm : listSt) { + ServerName sn = ProtobufUtil.toServerName(stm.getServerName()); + map.put(sn.getHostname() + ":" + sn.getPort(), stm.getTimestamp()); + } + } + return incrTimeRanges; + } + + private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) { + if (this.incrTimeRanges == null) { + return; + } + for (Entry> entry : this.incrTimeRanges.entrySet()) { + TableName key = entry.getKey(); + Map value = entry.getValue(); + BackupProtos.TableServerTimestamp.Builder tstBuilder = + BackupProtos.TableServerTimestamp.newBuilder(); + tstBuilder.setTableName(ProtobufUtil.toProtoTableName(key)); + + for (Map.Entry entry2 : value.entrySet()) { + String s = entry2.getKey(); + BackupProtos.ServerTimestamp.Builder stBuilder = + BackupProtos.ServerTimestamp.newBuilder(); + HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder(); + ServerName sn = ServerName.parseServerName(s); + snBuilder.setHostName(sn.getHostname()); + snBuilder.setPort(sn.getPort()); + stBuilder.setServerName(snBuilder.build()); + stBuilder.setTimestamp(entry2.getValue()); + tstBuilder.addServerTimestamp(stBuilder.build()); + } + builder.addTstMap(tstBuilder.build()); + } + } + + public String getBackupId() { + return backupId; + } + + private void setBackupId(String backupId) { + this.backupId = backupId; + } + + public BackupType getType() { + return type; + } + + private void setType(BackupType type) { + this.type = type; + } + + public String getRootDir() { + return rootDir; + } + + private void setRootDir(String rootDir) { + this.rootDir = rootDir; + } + + public List getTableNames() { + return tableList; + } + + private void setTableList(List tableList) { + this.tableList = tableList; + } + + public long getStartTs() { + return startTs; + } + + private void setStartTs(long startTs) { + this.startTs = startTs; + } + + public long getCompleteTs() { + return completeTs; + } + + private void setCompleteTs(long completeTs) { + this.completeTs = completeTs; + } + + public ArrayList getAncestors() { + if (this.ancestors == null) { + this.ancestors = new ArrayList<>(); + } + return this.ancestors; + } + + public void removeAncestors(List backupIds) { + List toRemove = new ArrayList<>(); + for (BackupImage im : this.ancestors) { + if (backupIds.contains(im.getBackupId())) { + toRemove.add(im); + } + } + this.ancestors.removeAll(toRemove); + } + + private void addAncestor(BackupImage backupImage) { + this.getAncestors().add(backupImage); + } + + public boolean hasAncestor(String token) { + for (BackupImage image : this.getAncestors()) { + if (image.getBackupId().equals(token)) { + return true; + } + } + return false; + } + + public boolean hasTable(TableName table) { + return tableList.contains(table); + } + + @Override + public int compareTo(BackupImage other) { + String thisBackupId = this.getBackupId(); + String otherBackupId = other.getBackupId(); + int index1 = thisBackupId.lastIndexOf("_"); + int index2 = otherBackupId.lastIndexOf("_"); + String name1 = thisBackupId.substring(0, index1); + String name2 = otherBackupId.substring(0, index2); + if (name1.equals(name2)) { + Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1)); + Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1)); + return thisTS.compareTo(otherTS); + } else { + return name1.compareTo(name2); + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BackupImage) { + return this.compareTo((BackupImage) obj) == 0; + } + return false; + } + + @Override + public int hashCode() { + int hash = 33 * this.getBackupId().hashCode() + type.hashCode(); + hash = 33 * hash + rootDir.hashCode(); + hash = 33 * hash + Long.valueOf(startTs).hashCode(); + hash = 33 * hash + Long.valueOf(completeTs).hashCode(); + for (TableName table : tableList) { + hash = 33 * hash + table.hashCode(); + } + return hash; + } + + public Map> getIncrTimeRanges() { + return incrTimeRanges; + } + + private void setIncrTimeRanges(Map> incrTimeRanges) { + this.incrTimeRanges = incrTimeRanges; + } + } + + // backup image directory + private BackupImage backupImage; + + /** + * Construct manifest for a ongoing backup. + * @param backup The ongoing backup info + */ + public BackupManifest(BackupInfo backup) { + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames()) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + } + + /** + * Construct a table level manifest for a backup of the named table. + * @param backup The ongoing backup session info + */ + public BackupManifest(BackupInfo backup, TableName table) { + List tables = new ArrayList(); + tables.add(table); + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(tables) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + } + + /** + * Construct manifest from a backup directory. + * @param conf configuration + * @param backupPath backup path + * @throws IOException if constructing the manifest from the backup directory fails + */ + public BackupManifest(Configuration conf, Path backupPath) throws IOException { + this(backupPath.getFileSystem(conf), backupPath); + } + + /** + * Construct manifest from a backup directory. + * @param fs the FileSystem + * @param backupPath backup path + * @throws BackupException exception + */ + public BackupManifest(FileSystem fs, Path backupPath) throws BackupException { + if (LOG.isDebugEnabled()) { + LOG.debug("Loading manifest from: " + backupPath.toString()); + } + // The input backupDir may not exactly be the backup table dir. + // It could be the backup log dir where there is also a manifest file stored. + // This variable's purpose is to keep the correct and original location so + // that we can store/persist it. + try { + FileStatus[] subFiles = BackupUtils.listStatus(fs, backupPath, null); + if (subFiles == null) { + String errorMsg = backupPath.toString() + " does not exist"; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + for (FileStatus subFile : subFiles) { + if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) { + // load and set manifest field from file content + long len = subFile.getLen(); + byte[] pbBytes = new byte[(int) len]; + try (FSDataInputStream in = fs.open(subFile.getPath())) { + in.readFully(pbBytes); + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + BackupProtos.BackupImage proto = null; + try { + proto = BackupProtos.BackupImage.parseFrom(pbBytes); + } catch (Exception e) { + throw new BackupException(e); + } + this.backupImage = BackupImage.fromProto(proto); + LOG.debug("Loaded manifest instance from manifest file: " + + BackupUtils.getPath(subFile.getPath())); + return; + } + } + String errorMsg = "No manifest file found in: " + backupPath.toString(); + throw new IOException(errorMsg); + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + } + + public BackupType getType() { + return backupImage.getType(); + } + + /** + * Get the table set of this image. + * @return The table set list + */ + public List getTableList() { + return backupImage.getTableNames(); + } + + /** + * TODO: fix it. Persist the manifest file. + * @throws BackupException if an error occurred while storing the manifest file. + */ + public void store(Configuration conf) throws BackupException { + byte[] data = backupImage.toProto().toByteArray(); + // write the file, overwrite if already exist + Path manifestFilePath = + new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(), backupImage.getBackupId()), + MANIFEST_FILE_NAME); + try (FSDataOutputStream out = + manifestFilePath.getFileSystem(conf).create(manifestFilePath, true)) { + out.write(data); + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + + LOG.info("Manifest file stored to " + manifestFilePath); + } + + /** + * Get this backup image. + * @return the backup image. + */ + public BackupImage getBackupImage() { + return backupImage; + } + + /** + * Add dependent backup image for this backup. + * @param image The direct dependent backup image + */ + public void addDependentImage(BackupImage image) { + this.backupImage.addAncestor(image); + } + + /** + * Set the incremental timestamp map directly. + * @param incrTimestampMap timestamp map + */ + public void setIncrTimestampMap(Map> incrTimestampMap) { + this.backupImage.setIncrTimeRanges(incrTimestampMap); + } + + public Map> getIncrTimestampMap() { + return backupImage.getIncrTimeRanges(); + } + + /** + * Get the image list of this backup for restore in time order. + * @param reverse If true, then output in reverse order, otherwise in time order from old to new + * @return the backup image list for restore in time order + */ + public ArrayList getRestoreDependentList(boolean reverse) { + TreeMap restoreImages = new TreeMap<>(); + restoreImages.put(backupImage.startTs, backupImage); + for (BackupImage image : backupImage.getAncestors()) { + restoreImages.put(Long.valueOf(image.startTs), image); + } + return new ArrayList<>( + reverse ? restoreImages.descendingMap().values() : restoreImages.values()); + } + + /** + * Get the dependent image list for a specific table of this backup in time order from old to new + * if want to restore to this backup image level. + * @param table table + * @return the backup image list for a table in time order + */ + public ArrayList getDependentListByTable(TableName table) { + ArrayList tableImageList = new ArrayList<>(); + ArrayList imageList = getRestoreDependentList(true); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + if (image.getType() == BackupType.FULL) { + break; + } + } + } + Collections.reverse(tableImageList); + return tableImageList; + } + + /** + * Get the full dependent image list in the whole dependency scope for a specific table of this + * backup in time order from old to new. + * @param table table + * @return the full backup image list for a table in time order in the whole scope of the + * dependency of this image + */ + public ArrayList getAllDependentListByTable(TableName table) { + ArrayList tableImageList = new ArrayList<>(); + ArrayList imageList = getRestoreDependentList(false); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + } + } + return tableImageList; + } + + /** + * Check whether backup image1 could cover backup image2 or not. + * @param image1 backup image 1 + * @param image2 backup image 2 + * @return true if image1 can cover image2, otherwise false + */ + public static boolean canCoverImage(BackupImage image1, BackupImage image2) { + // image1 can cover image2 only when the following conditions are satisfied: + // - image1 must not be an incremental image; + // - image1 must be taken after image2 has been taken; + // - table set of image1 must cover the table set of image2. + if (image1.getType() == BackupType.INCREMENTAL) { + return false; + } + if (image1.getStartTs() < image2.getStartTs()) { + return false; + } + List image1TableList = image1.getTableNames(); + List image2TableList = image2.getTableNames(); + boolean found; + for (int i = 0; i < image2TableList.size(); i++) { + found = false; + for (int j = 0; j < image1TableList.size(); j++) { + if (image2TableList.get(i).equals(image1TableList.get(j))) { + found = true; + break; + } + } + if (!found) { + return false; + } + } + + LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId()); + return true; + } + + /** + * Check whether backup image set could cover a backup image or not. + * @param fullImages The backup image set + * @param image The target backup image + * @return true if fullImages can cover image, otherwise false + */ + public static boolean canCoverImage(ArrayList fullImages, BackupImage image) { + // fullImages can cover image only when the following conditions are satisfied: + // - each image of fullImages must not be an incremental image; + // - each image of fullImages must be taken after image has been taken; + // - sum table set of fullImages must cover the table set of image. + for (BackupImage image1 : fullImages) { + if (image1.getType() == BackupType.INCREMENTAL) { + return false; + } + if (image1.getStartTs() < image.getStartTs()) { + return false; + } + } + + ArrayList image1TableList = new ArrayList<>(); + for (BackupImage image1 : fullImages) { + List tableList = image1.getTableNames(); + for (TableName table : tableList) { + image1TableList.add(table.getNameAsString()); + } + } + ArrayList image2TableList = new ArrayList<>(); + List tableList = image.getTableNames(); + for (TableName table : tableList) { + image2TableList.add(table.getNameAsString()); + } + + for (int i = 0; i < image2TableList.size(); i++) { + if (image1TableList.contains(image2TableList.get(i)) == false) { + return false; + } + } + + LOG.debug("Full image set can cover image " + image.getBackupId()); + return true; + } + + public BackupInfo toBackupInfo() { + BackupInfo info = new BackupInfo(); + info.setType(backupImage.getType()); + List list = backupImage.getTableNames(); + TableName[] tables = new TableName[list.size()]; + info.addTables(list.toArray(tables)); + info.setBackupId(backupImage.getBackupId()); + info.setStartTs(backupImage.getStartTs()); + info.setBackupRootDir(backupImage.getRootDir()); + if (backupImage.getType() == BackupType.INCREMENTAL) { + info.setHLogTargetDir( + BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId())); + } + return info; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java new file mode 100644 index 000000000000..1716a3ff293d --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -0,0 +1,1877 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterators; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + +/** + * This class provides API to access backup system table
+ * Backup system table schema:
+ *

+ *

    + *
  • 1. Backup sessions rowkey= "session:"+backupId; value =serialized BackupInfo
  • + *
  • 2. Backup start code rowkey = "startcode:"+backupRoot; value = startcode
  • + *
  • 3. Incremental backup set rowkey="incrbackupset:"+backupRoot; value=[list of tables]
  • + *
  • 4. Table-RS-timestamp map rowkey="trslm:"+backupRoot+table_name; value = map[RS-%3E last WAL + * timestamp]
  • + *
  • 5. RS - WAL ts map rowkey="rslogts:"+backupRoot +server; value = last WAL timestamp
  • + *
  • 6. WALs recorded rowkey="wals:"+WAL unique file name; value = backupId and full WAL file + * name
  • + *
+ *

+ */ +@InterfaceAudience.Private +public final class BackupSystemTable implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(BackupSystemTable.class); + + static class WALItem { + String backupId; + String walFile; + String backupRoot; + + WALItem(String backupId, String walFile, String backupRoot) { + this.backupId = backupId; + this.walFile = walFile; + this.backupRoot = backupRoot; + } + + public String getBackupId() { + return backupId; + } + + public String getWalFile() { + return walFile; + } + + public String getBackupRoot() { + return backupRoot; + } + + @Override + public String toString() { + return Path.SEPARATOR + backupRoot + Path.SEPARATOR + backupId + Path.SEPARATOR + walFile; + } + } + + /** + * Backup system table (main) name + */ + private TableName tableName; + + /** + * Backup System table name for bulk loaded files. We keep all bulk loaded file references in a + * separate table because we have to isolate general backup operations: create, merge etc from + * activity of RegionObserver, which controls process of a bulk loading + * {@link org.apache.hadoop.hbase.backup.BackupObserver} + */ + private TableName bulkLoadTableName; + + /** + * Stores backup sessions (contexts) + */ + final static byte[] SESSIONS_FAMILY = Bytes.toBytes("session"); + /** + * Stores other meta + */ + final static byte[] META_FAMILY = Bytes.toBytes("meta"); + final static byte[] BULK_LOAD_FAMILY = Bytes.toBytes("bulk"); + /** + * Connection to HBase cluster, shared among all instances + */ + private final Connection connection; + + private final static String BACKUP_INFO_PREFIX = "session:"; + private final static String START_CODE_ROW = "startcode:"; + private final static byte[] ACTIVE_SESSION_ROW = Bytes.toBytes("activesession:"); + private final static byte[] ACTIVE_SESSION_COL = Bytes.toBytes("c"); + + private final static byte[] ACTIVE_SESSION_YES = Bytes.toBytes("yes"); + private final static byte[] ACTIVE_SESSION_NO = Bytes.toBytes("no"); + + private final static String INCR_BACKUP_SET = "incrbackupset:"; + private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:"; + private final static String RS_LOG_TS_PREFIX = "rslogts:"; + + private final static String BULK_LOAD_PREFIX = "bulk:"; + private final static byte[] BULK_LOAD_PREFIX_BYTES = Bytes.toBytes(BULK_LOAD_PREFIX); + private final static byte[] DELETE_OP_ROW = Bytes.toBytes("delete_op_row"); + private final static byte[] MERGE_OP_ROW = Bytes.toBytes("merge_op_row"); + + final static byte[] TBL_COL = Bytes.toBytes("tbl"); + final static byte[] FAM_COL = Bytes.toBytes("fam"); + final static byte[] PATH_COL = Bytes.toBytes("path"); + final static byte[] STATE_COL = Bytes.toBytes("state"); + // the two states a bulk loaded file can be + final static byte[] BL_PREPARE = Bytes.toBytes("R"); + final static byte[] BL_COMMIT = Bytes.toBytes("D"); + + private final static String SET_KEY_PREFIX = "backupset:"; + + // separator between BULK_LOAD_PREFIX and ordinals + private final static String BLK_LD_DELIM = ":"; + private final static byte[] EMPTY_VALUE = new byte[] {}; + + // Safe delimiter in a string + private final static String NULL = "\u0000"; + + public BackupSystemTable(Connection conn) throws IOException { + this.connection = conn; + Configuration conf = this.connection.getConfiguration(); + tableName = BackupSystemTable.getTableName(conf); + bulkLoadTableName = BackupSystemTable.getTableNameForBulkLoadedData(conf); + checkSystemTable(); + } + + private void checkSystemTable() throws IOException { + try (Admin admin = connection.getAdmin()) { + verifyNamespaceExists(admin); + Configuration conf = connection.getConfiguration(); + if (!admin.tableExists(tableName)) { + TableDescriptor backupHTD = BackupSystemTable.getSystemTableDescriptor(conf); + admin.createTable(backupHTD); + } + if (!admin.tableExists(bulkLoadTableName)) { + TableDescriptor blHTD = BackupSystemTable.getSystemTableForBulkLoadedDataDescriptor(conf); + admin.createTable(blHTD); + } + waitForSystemTable(admin, tableName); + waitForSystemTable(admin, bulkLoadTableName); + } + } + + private void verifyNamespaceExists(Admin admin) throws IOException { + String namespaceName = tableName.getNamespaceAsString(); + NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build(); + NamespaceDescriptor[] list = admin.listNamespaceDescriptors(); + boolean exists = false; + for (NamespaceDescriptor nsd : list) { + if (nsd.getName().equals(ns.getName())) { + exists = true; + break; + } + } + if (!exists) { + admin.createNamespace(ns); + } + } + + private void waitForSystemTable(Admin admin, TableName tableName) throws IOException { + // Return fast if the table is available and avoid a log message + if (admin.tableExists(tableName) && admin.isTableAvailable(tableName)) { + return; + } + long TIMEOUT = 60000; + long startTime = EnvironmentEdgeManager.currentTime(); + LOG.debug("Backup table {} is not present and available, waiting for it to become so", + tableName); + while (!admin.tableExists(tableName) || !admin.isTableAvailable(tableName)) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw (IOException) new InterruptedIOException().initCause(e); + } + if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) { + throw new IOException( + "Failed to create backup system table " + tableName + " after " + TIMEOUT + "ms"); + } + } + LOG.debug("Backup table {} exists and available", tableName); + } + + @Override + public void close() { + // do nothing + } + + /** + * Updates status (state) of a backup session in backup system table table + * @param info backup info + * @throws IOException exception + */ + public void updateBackupInfo(BackupInfo info) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("update backup status in backup system table for: " + info.getBackupId() + + " set status=" + info.getState()); + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForBackupInfo(info); + table.put(put); + } + } + + /* + * @param backupId the backup Id + * @return Map of rows to path of bulk loaded hfile + */ + Map readBulkLoadedFiles(String backupId) throws IOException { + Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId); + try (Table table = connection.getTable(bulkLoadTableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res = null; + Map map = new TreeMap<>(Bytes.BYTES_COMPARATOR); + while ((res = scanner.next()) != null) { + res.advance(); + byte[] row = CellUtil.cloneRow(res.listCells().get(0)); + for (Cell cell : res.listCells()) { + if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0, + BackupSystemTable.PATH_COL.length) == 0 + ) { + map.put(row, Bytes.toString(CellUtil.cloneValue(cell))); + } + } + } + return map; + } + } + + /* + * Used during restore + * @param backupId the backup Id + * @param sTableList List of tables + * @return array of Map of family to List of Paths + */ + public Map>[] readBulkLoadedFiles(String backupId, List sTableList) + throws IOException { + Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId); + @SuppressWarnings("unchecked") + Map>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()]; + try (Table table = connection.getTable(bulkLoadTableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + TableName tbl = null; + byte[] fam = null; + String path = null; + for (Cell cell : res.listCells()) { + if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0, + BackupSystemTable.TBL_COL.length) == 0 + ) { + tbl = TableName.valueOf(CellUtil.cloneValue(cell)); + } else if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0, + BackupSystemTable.FAM_COL.length) == 0 + ) { + fam = CellUtil.cloneValue(cell); + } else if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0, + BackupSystemTable.PATH_COL.length) == 0 + ) { + path = Bytes.toString(CellUtil.cloneValue(cell)); + } + } + int srcIdx = IncrementalTableBackupClient.getIndex(tbl, sTableList); + if (srcIdx == -1) { + // the table is not among the query + continue; + } + if (mapForSrc[srcIdx] == null) { + mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR); + } + List files; + if (!mapForSrc[srcIdx].containsKey(fam)) { + files = new ArrayList(); + mapForSrc[srcIdx].put(fam, files); + } else { + files = mapForSrc[srcIdx].get(fam); + } + files.add(new Path(path)); + if (LOG.isDebugEnabled()) { + LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path); + } + } + + return mapForSrc; + } + } + + /** + * Deletes backup status from backup system table table + * @param backupId backup id + * @throws IOException exception + */ + public void deleteBackupInfo(String backupId) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("delete backup status in backup system table for " + backupId); + } + try (Table table = connection.getTable(tableName)) { + Delete del = createDeleteForBackupInfo(backupId); + table.delete(del); + } + } + + /* + * For postBulkLoadHFile() hook. + * @param tabName table name + * @param region the region receiving hfile + * @param finalPaths family and associated hfiles + */ + public void writePathsPostBulkLoad(TableName tabName, byte[] region, + Map> finalPaths) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size() + + " entries"); + } + try (Table table = connection.getTable(bulkLoadTableName)) { + List puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths); + table.put(puts); + LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName); + } + } + + /* + * For preCommitStoreFile() hook + * @param tabName table name + * @param region the region receiving hfile + * @param family column family + * @param pairs list of paths for hfiles + */ + public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family, + final List> pairs) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug( + "write bulk load descriptor to backup " + tabName + " with " + pairs.size() + " entries"); + } + try (Table table = connection.getTable(bulkLoadTableName)) { + List puts = + BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs); + table.put(puts); + LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName); + } + } + + /* + * Removes rows recording bulk loaded hfiles from backup table + * @param lst list of table names + * @param rows the rows to be deleted + */ + public void deleteBulkLoadedRows(List rows) throws IOException { + try (Table table = connection.getTable(bulkLoadTableName)) { + List lstDels = new ArrayList<>(); + for (byte[] row : rows) { + Delete del = new Delete(row); + lstDels.add(del); + LOG.debug("orig deleting the row: " + Bytes.toString(row)); + } + table.delete(lstDels); + LOG.debug("deleted " + rows.size() + " original bulkload rows"); + } + } + + /* + * Reads the rows from backup table recording bulk loaded hfiles + * @param tableList list of table names + * @return The keys of the Map are table, region and column family. Value of the map reflects + * whether the hfile was recorded by preCommitStoreFile hook (true) + */ + public Pair>>>>, List> + readBulkloadRows(List tableList) throws IOException { + + Map>>>> map = new HashMap<>(); + List rows = new ArrayList<>(); + for (TableName tTable : tableList) { + Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(tTable); + Map>>> tblMap = map.get(tTable); + try (Table table = connection.getTable(bulkLoadTableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + String fam = null; + String path = null; + boolean raw = false; + byte[] row; + String region = null; + for (Cell cell : res.listCells()) { + row = CellUtil.cloneRow(cell); + rows.add(row); + String rowStr = Bytes.toString(row); + region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr); + if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0, + BackupSystemTable.FAM_COL.length) == 0 + ) { + fam = Bytes.toString(CellUtil.cloneValue(cell)); + } else if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0, + BackupSystemTable.PATH_COL.length) == 0 + ) { + path = Bytes.toString(CellUtil.cloneValue(cell)); + } else if ( + CellUtil.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0, + BackupSystemTable.STATE_COL.length) == 0 + ) { + byte[] state = CellUtil.cloneValue(cell); + if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) { + raw = true; + } else { + raw = false; + } + } + } + if (map.get(tTable) == null) { + map.put(tTable, new HashMap<>()); + tblMap = map.get(tTable); + } + if (tblMap.get(region) == null) { + tblMap.put(region, new HashMap<>()); + } + Map>> famMap = tblMap.get(region); + if (famMap.get(fam) == null) { + famMap.put(fam, new ArrayList<>()); + } + famMap.get(fam).add(new Pair<>(path, raw)); + LOG.debug("found orig " + path + " for " + fam + " of table " + region); + } + } + } + return new Pair<>(map, rows); + } + + /* + * @param sTableList List of tables + * @param maps array of Map of family to List of Paths + * @param backupId the backup Id + */ + public void writeBulkLoadedFiles(List sTableList, Map>[] maps, + String backupId) throws IOException { + try (Table table = connection.getTable(bulkLoadTableName)) { + long ts = EnvironmentEdgeManager.currentTime(); + int cnt = 0; + List puts = new ArrayList<>(); + for (int idx = 0; idx < maps.length; idx++) { + Map> map = maps[idx]; + TableName tn = sTableList.get(idx); + + if (map == null) { + continue; + } + + for (Map.Entry> entry : map.entrySet()) { + byte[] fam = entry.getKey(); + List paths = entry.getValue(); + for (Path p : paths) { + Put put = BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, + ts, cnt++); + puts.add(put); + } + } + } + if (!puts.isEmpty()) { + table.put(puts); + } + } + } + + /** + * Reads backup status object (instance of backup info) from backup system table table + * @param backupId backup id + * @return Current status of backup session or null + */ + public BackupInfo readBackupInfo(String backupId) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read backup status from backup system table for: " + backupId); + } + + try (Table table = connection.getTable(tableName)) { + Get get = createGetForBackupInfo(backupId); + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + return resultToBackupInfo(res); + } + } + + /** + * Read the last backup start code (timestamp) of last successful backup. Will return null if + * there is no start code stored on hbase or the value is of length 0. These two cases indicate + * there is no successful backup completed so far. + * @param backupRoot directory path to backup destination + * @return the timestamp of last successful backup + * @throws IOException exception + */ + public String readBackupStartCode(String backupRoot) throws IOException { + LOG.trace("read backup start code from backup system table"); + + try (Table table = connection.getTable(tableName)) { + Get get = createGetForStartCode(backupRoot); + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + Cell cell = res.listCells().get(0); + byte[] val = CellUtil.cloneValue(cell); + if (val.length == 0) { + return null; + } + return new String(val, StandardCharsets.UTF_8); + } + } + + /** + * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte. + * @param startCode start code + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("write backup start code to backup system table " + startCode); + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForStartCode(startCode.toString(), backupRoot); + table.put(put); + } + } + + /** + * Exclusive operations are: create, delete, merge + * @throws IOException if a table operation fails or an active backup exclusive operation is + * already underway + */ + public void startBackupExclusiveOperation() throws IOException { + LOG.debug("Start new backup exclusive operation"); + + try (Table table = connection.getTable(tableName)) { + Put put = createPutForStartBackupSession(); + // First try to put if row does not exist + if ( + !table.checkAndMutate(ACTIVE_SESSION_ROW, SESSIONS_FAMILY).qualifier(ACTIVE_SESSION_COL) + .ifNotExists().thenPut(put) + ) { + // Row exists, try to put if value == ACTIVE_SESSION_NO + if ( + !table.checkAndMutate(ACTIVE_SESSION_ROW, SESSIONS_FAMILY).qualifier(ACTIVE_SESSION_COL) + .ifEquals(ACTIVE_SESSION_NO).thenPut(put) + ) { + throw new ExclusiveOperationException(); + } + } + } + } + + private Put createPutForStartBackupSession() { + Put put = new Put(ACTIVE_SESSION_ROW); + put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_YES); + return put; + } + + public void finishBackupExclusiveOperation() throws IOException { + LOG.debug("Finish backup exclusive operation"); + + try (Table table = connection.getTable(tableName)) { + Put put = createPutForStopBackupSession(); + if ( + !table.checkAndMutate(ACTIVE_SESSION_ROW, SESSIONS_FAMILY).qualifier(ACTIVE_SESSION_COL) + .ifEquals(ACTIVE_SESSION_YES).thenPut(put) + ) { + throw new IOException("There is no active backup exclusive operation"); + } + } + } + + private Put createPutForStopBackupSession() { + Put put = new Put(ACTIVE_SESSION_ROW); + put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_NO); + return put; + } + + /** + * Get the Region Servers log information after the last log roll from backup system table. + * @param backupRoot root directory path to backup + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult(String backupRoot) + throws IOException { + LOG.trace("read region server last roll log result to backup system table"); + + Scan scan = createScanForReadRegionServerLastLogRollResult(backupRoot); + + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res; + HashMap rsTimestampMap = new HashMap<>(); + while ((res = scanner.next()) != null) { + res.advance(); + Cell cell = res.current(); + byte[] row = CellUtil.cloneRow(cell); + String server = getServerNameForReadRegionServerLastLogRollResult(row); + byte[] data = CellUtil.cloneValue(cell); + rsTimestampMap.put(server, Bytes.toLong(data)); + } + return rsTimestampMap; + } + } + + /** + * Writes Region Server last roll log result (timestamp) to backup system table table + * @param server Region Server name + * @param ts last log timestamp + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot) + throws IOException { + LOG.trace("write region server last roll log result to backup system table"); + + try (Table table = connection.getTable(tableName)) { + Put put = createPutForRegionServerLastLogRollResult(server, ts, backupRoot); + table.put(put); + } + } + + /** + * Get all completed backup information (in desc order by time) + * @param onlyCompleted true, if only successfully completed sessions + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public ArrayList getBackupHistory(boolean onlyCompleted) throws IOException { + LOG.trace("get backup history from backup system table"); + + BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY; + ArrayList list = getBackupInfos(state); + return BackupUtils.sortHistoryListDesc(list); + } + + /** + * Get all backups history + * @return list of backup info + * @throws IOException if getting the backup history fails + */ + public List getBackupHistory() throws IOException { + return getBackupHistory(false); + } + + /** + * Get first n backup history records + * @param n number of records, if n== -1 - max number is ignored + * @return list of records + * @throws IOException if getting the backup history fails + */ + public List getHistory(int n) throws IOException { + List history = getBackupHistory(); + if (n == -1 || history.size() <= n) { + return history; + } + return Collections.unmodifiableList(history.subList(0, n)); + } + + /** + * Get backup history records filtered by list of filters. + * @param n max number of records, if n == -1 , then max number is ignored + * @param filters list of filters + * @return backup records + * @throws IOException if getting the backup history fails + */ + public List getBackupHistory(int n, BackupInfo.Filter... filters) throws IOException { + if (filters.length == 0) { + return getHistory(n); + } + + List history = getBackupHistory(); + List result = new ArrayList<>(); + for (BackupInfo bi : history) { + if (n >= 0 && result.size() == n) { + break; + } + + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(bi)) { + passed = false; + break; + } + } + if (passed) { + result.add(bi); + } + } + return result; + } + + /* + * Retrieve TableName's for completed backup of given type + * @param type backup type + * @return List of table names + */ + public List getTablesForBackupType(BackupType type) throws IOException { + Set names = new HashSet<>(); + List infos = getBackupHistory(true); + for (BackupInfo info : infos) { + if (info.getType() == type) { + names.addAll(info.getTableNames()); + } + } + return new ArrayList<>(names); + } + + /** + * Get history for backup destination + * @param backupRoot backup destination path + * @return List of backup info + * @throws IOException if getting the backup history fails + */ + public List getBackupHistory(String backupRoot) throws IOException { + ArrayList history = getBackupHistory(false); + for (Iterator iterator = history.iterator(); iterator.hasNext();) { + BackupInfo info = iterator.next(); + if (!backupRoot.equals(info.getBackupRootDir())) { + iterator.remove(); + } + } + return history; + } + + /** + * Get history for a table + * @param name table name + * @return history for a table + * @throws IOException if getting the backup history fails + */ + public List getBackupHistoryForTable(TableName name) throws IOException { + List history = getBackupHistory(); + List tableHistory = new ArrayList<>(); + for (BackupInfo info : history) { + List tables = info.getTableNames(); + if (tables.contains(name)) { + tableHistory.add(info); + } + } + return tableHistory; + } + + public Map> getBackupHistoryForTableSet(Set set, + String backupRoot) throws IOException { + List history = getBackupHistory(backupRoot); + Map> tableHistoryMap = new HashMap<>(); + for (Iterator iterator = history.iterator(); iterator.hasNext();) { + BackupInfo info = iterator.next(); + if (!backupRoot.equals(info.getBackupRootDir())) { + continue; + } + List tables = info.getTableNames(); + for (TableName tableName : tables) { + if (set.contains(tableName)) { + ArrayList list = tableHistoryMap.get(tableName); + if (list == null) { + list = new ArrayList<>(); + tableHistoryMap.put(tableName, list); + } + list.add(info); + } + } + } + return tableHistoryMap; + } + + /** + * Get all backup sessions with a given state (in descending order by time) + * @param state backup session state + * @return history info of backup info objects + * @throws IOException exception + */ + public ArrayList getBackupInfos(BackupState state) throws IOException { + LOG.trace("get backup infos from backup system table"); + + Scan scan = createScanForBackupHistory(); + ArrayList list = new ArrayList<>(); + + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res; + while ((res = scanner.next()) != null) { + res.advance(); + BackupInfo context = cellToBackupInfo(res.current()); + if (state != BackupState.ANY && context.getState() != state) { + continue; + } + list.add(context); + } + return list; + } + } + + /** + * Write the current timestamps for each regionserver to backup system table after a successful + * full or incremental backup. The saved timestamp is of the last log file that was backed up + * already. + * @param tables tables + * @param newTimestamps timestamps + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, Map newTimestamps, + String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("write RS log time stamps to backup system table for tables [" + + StringUtils.join(tables, ",") + "]"); + } + List puts = new ArrayList<>(); + for (TableName table : tables) { + byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray(); + Put put = createPutForWriteRegionServerLogTimestamp(table, smapData, backupRoot); + puts.add(put); + } + try (Table table = connection.getTable(tableName)) { + table.put(puts); + } + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. The info is stored for each table as a concatenated string of + * rs-%3Etimestapmp + * @param backupRoot root directory path to backup + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public Map> readLogTimestampMap(String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read RS log ts from backup system table for root=" + backupRoot); + } + + Map> tableTimestampMap = new HashMap<>(); + + Scan scan = createScanForReadLogTimestampMap(backupRoot); + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res; + while ((res = scanner.next()) != null) { + res.advance(); + Cell cell = res.current(); + byte[] row = CellUtil.cloneRow(cell); + String tabName = getTableNameForReadLogTimestampMap(row); + TableName tn = TableName.valueOf(tabName); + byte[] data = CellUtil.cloneValue(cell); + if (data == null) { + throw new IOException("Data of last backup data from backup system table " + + "is empty. Create a backup first."); + } + if (data != null && data.length > 0) { + HashMap lastBackup = + fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data)); + tableTimestampMap.put(tn, lastBackup); + } + } + return tableTimestampMap; + } + } + + private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table, + Map map) { + BackupProtos.TableServerTimestamp.Builder tstBuilder = + BackupProtos.TableServerTimestamp.newBuilder(); + tstBuilder + .setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toProtoTableName(table)); + + for (Entry entry : map.entrySet()) { + BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder(); + HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder(); + ServerName sn = ServerName.parseServerName(entry.getKey()); + snBuilder.setHostName(sn.getHostname()); + snBuilder.setPort(sn.getPort()); + builder.setServerName(snBuilder.build()); + builder.setTimestamp(entry.getValue()); + tstBuilder.addServerTimestamp(builder.build()); + } + + return tstBuilder.build(); + } + + private HashMap + fromTableServerTimestampProto(BackupProtos.TableServerTimestamp proto) { + + HashMap map = new HashMap<>(); + List list = proto.getServerTimestampList(); + for (BackupProtos.ServerTimestamp st : list) { + ServerName sn = + org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(st.getServerName()); + map.put(sn.getHostname() + ":" + sn.getPort(), st.getTimestamp()); + } + return map; + } + + /** + * Return the current tables covered by incremental backup. + * @param backupRoot root directory path to backup + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet(String backupRoot) throws IOException { + LOG.trace("get incremental backup table set from backup system table"); + + TreeSet set = new TreeSet<>(); + + try (Table table = connection.getTable(tableName)) { + Get get = createGetForIncrBackupTableSet(backupRoot); + Result res = table.get(get); + if (res.isEmpty()) { + return set; + } + List cells = res.listCells(); + for (Cell cell : cells) { + // qualifier = table name - we use table names as qualifiers + set.add(TableName.valueOf(CellUtil.cloneQualifier(cell))); + } + return set; + } + } + + /** + * Add tables to global incremental backup set + * @param tables set of tables + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables, String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Add incremental backup table set to backup system table. ROOT=" + backupRoot + + " tables [" + StringUtils.join(tables, " ") + "]"); + } + if (LOG.isDebugEnabled()) { + tables.forEach(table -> LOG.debug(Objects.toString(table))); + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForIncrBackupTableSet(tables, backupRoot); + table.put(put); + } + } + + /** + * Deletes incremental backup set for a backup destination + * @param backupRoot backup root + */ + public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Delete incremental backup table set to backup system table. ROOT=" + backupRoot); + } + try (Table table = connection.getTable(tableName)) { + Delete delete = createDeleteForIncrBackupTableSet(backupRoot); + table.delete(delete); + } + } + + /** + * Checks if we have at least one backup session in backup system table This API is used by + * BackupLogCleaner + * @return true, if - at least one session exists in backup system table table + * @throws IOException exception + */ + public boolean hasBackupSessions() throws IOException { + LOG.trace("Has backup sessions from backup system table"); + + boolean result = false; + Scan scan = createScanForBackupHistory(); + scan.setCaching(1); + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + if (scanner.next() != null) { + result = true; + } + return result; + } + } + + /** + * BACKUP SETS + */ + + /** + * Get backup set list + * @return backup set list + * @throws IOException if a table or scanner operation fails + */ + public List listBackupSets() throws IOException { + LOG.trace("Backup set list"); + + List list = new ArrayList<>(); + try (Table table = connection.getTable(tableName)) { + Scan scan = createScanForBackupSetList(); + scan.readVersions(1); + try (ResultScanner scanner = table.getScanner(scan)) { + Result res; + while ((res = scanner.next()) != null) { + res.advance(); + list.add(cellKeyToBackupSetName(res.current())); + } + return list; + } + } + } + + /** + * Get backup set description (list of tables) + * @param name set's name + * @return list of tables in a backup set + * @throws IOException if a table operation fails + */ + public List describeBackupSet(String name) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set describe: " + name); + } + try (Table table = connection.getTable(tableName)) { + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + res.advance(); + String[] tables = cellValueToBackupSet(res.current()); + return Arrays.asList(tables).stream().map(item -> TableName.valueOf(item)) + .collect(Collectors.toList()); + } + } + + /** + * Add backup set (list of tables) + * @param name set name + * @param newTables list of tables, comma-separated + * @throws IOException if a table operation fails + */ + public void addToBackupSet(String name, String[] newTables) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]"); + } + String[] union = null; + try (Table table = connection.getTable(tableName)) { + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + union = newTables; + } else { + res.advance(); + String[] tables = cellValueToBackupSet(res.current()); + union = merge(tables, newTables); + } + Put put = createPutForBackupSet(name, union); + table.put(put); + } + } + + /** + * Remove tables from backup set (list of tables) + * @param name set name + * @param toRemove list of tables + * @throws IOException if a table operation or deleting the backup set fails + */ + public void removeFromBackupSet(String name, String[] toRemove) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace( + " Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ") + "]"); + } + String[] disjoint; + String[] tables; + try (Table table = connection.getTable(tableName)) { + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + LOG.warn("Backup set '" + name + "' not found."); + return; + } else { + res.advance(); + tables = cellValueToBackupSet(res.current()); + disjoint = disjoin(tables, toRemove); + } + if (disjoint.length > 0 && disjoint.length != tables.length) { + Put put = createPutForBackupSet(name, disjoint); + table.put(put); + } else if (disjoint.length == tables.length) { + LOG.warn("Backup set '" + name + "' does not contain tables [" + + StringUtils.join(toRemove, " ") + "]"); + } else { // disjoint.length == 0 and tables.length >0 + // Delete backup set + LOG.info("Backup set '" + name + "' is empty. Deleting."); + deleteBackupSet(name); + } + } + } + + private String[] merge(String[] existingTables, String[] newTables) { + Set tables = new HashSet<>(Arrays.asList(existingTables)); + tables.addAll(Arrays.asList(newTables)); + return tables.toArray(new String[0]); + } + + private String[] disjoin(String[] existingTables, String[] toRemove) { + Set tables = new HashSet<>(Arrays.asList(existingTables)); + Arrays.asList(toRemove).forEach(table -> tables.remove(table)); + return tables.toArray(new String[0]); + } + + /** + * Delete backup set + * @param name set's name + * @throws IOException if getting or deleting the table fails + */ + public void deleteBackupSet(String name) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set delete: " + name); + } + try (Table table = connection.getTable(tableName)) { + Delete del = createDeleteForBackupSet(name); + table.delete(del); + } + } + + /** + * Get backup system table descriptor + * @return table's descriptor + */ + public static TableDescriptor getSystemTableDescriptor(Configuration conf) { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(getTableName(conf)); + + ColumnFamilyDescriptorBuilder colBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(SESSIONS_FAMILY); + + colBuilder.setMaxVersions(1); + Configuration config = HBaseConfiguration.create(); + int ttl = config.getInt(BackupRestoreConstants.BACKUP_SYSTEM_TTL_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TTL_DEFAULT); + colBuilder.setTimeToLive(ttl); + + ColumnFamilyDescriptor colSessionsDesc = colBuilder.build(); + builder.setColumnFamily(colSessionsDesc); + + colBuilder = ColumnFamilyDescriptorBuilder.newBuilder(META_FAMILY); + colBuilder.setTimeToLive(ttl); + builder.setColumnFamily(colBuilder.build()); + return builder.build(); + } + + public static TableName getTableName(Configuration conf) { + String name = conf.get(BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_DEFAULT); + return TableName.valueOf(name); + } + + public static String getTableNameAsString(Configuration conf) { + return getTableName(conf).getNameAsString(); + } + + public static String getSnapshotName(Configuration conf) { + return "snapshot_" + getTableNameAsString(conf).replace(":", "_"); + } + + /** + * Get backup system table descriptor + * @return table's descriptor + */ + public static TableDescriptor getSystemTableForBulkLoadedDataDescriptor(Configuration conf) { + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(getTableNameForBulkLoadedData(conf)); + + ColumnFamilyDescriptorBuilder colBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(SESSIONS_FAMILY); + colBuilder.setMaxVersions(1); + Configuration config = HBaseConfiguration.create(); + int ttl = config.getInt(BackupRestoreConstants.BACKUP_SYSTEM_TTL_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TTL_DEFAULT); + colBuilder.setTimeToLive(ttl); + ColumnFamilyDescriptor colSessionsDesc = colBuilder.build(); + builder.setColumnFamily(colSessionsDesc); + colBuilder = ColumnFamilyDescriptorBuilder.newBuilder(META_FAMILY); + colBuilder.setTimeToLive(ttl); + builder.setColumnFamily(colBuilder.build()); + return builder.build(); + } + + public static TableName getTableNameForBulkLoadedData(Configuration conf) { + String name = conf.get(BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_DEFAULT) + "_bulk"; + return TableName.valueOf(name); + } + + /** + * Creates Put operation for a given backup info object + * @param context backup info + * @return put operation + * @throws IOException exception + */ + private Put createPutForBackupInfo(BackupInfo context) throws IOException { + Put put = new Put(rowkey(BACKUP_INFO_PREFIX, context.getBackupId())); + put.addColumn(BackupSystemTable.SESSIONS_FAMILY, Bytes.toBytes("context"), + context.toByteArray()); + return put; + } + + /** + * Creates Get operation for a given backup id + * @param backupId backup's ID + * @return get operation + * @throws IOException exception + */ + private Get createGetForBackupInfo(String backupId) throws IOException { + Get get = new Get(rowkey(BACKUP_INFO_PREFIX, backupId)); + get.addFamily(BackupSystemTable.SESSIONS_FAMILY); + get.readVersions(1); + return get; + } + + /** + * Creates Delete operation for a given backup id + * @param backupId backup's ID + * @return delete operation + */ + private Delete createDeleteForBackupInfo(String backupId) { + Delete del = new Delete(rowkey(BACKUP_INFO_PREFIX, backupId)); + del.addFamily(BackupSystemTable.SESSIONS_FAMILY); + return del; + } + + /** + * Converts Result to BackupInfo + * @param res HBase result + * @return backup info instance + * @throws IOException exception + */ + private BackupInfo resultToBackupInfo(Result res) throws IOException { + res.advance(); + Cell cell = res.current(); + return cellToBackupInfo(cell); + } + + /** + * Creates Get operation to retrieve start code from backup system table + * @return get operation + * @throws IOException exception + */ + private Get createGetForStartCode(String rootPath) throws IOException { + Get get = new Get(rowkey(START_CODE_ROW, rootPath)); + get.addFamily(BackupSystemTable.META_FAMILY); + get.readVersions(1); + return get; + } + + /** + * Creates Put operation to store start code to backup system table + * @return put operation + */ + private Put createPutForStartCode(String startCode, String rootPath) { + Put put = new Put(rowkey(START_CODE_ROW, rootPath)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("startcode"), + Bytes.toBytes(startCode)); + return put; + } + + /** + * Creates Get to retrieve incremental backup table set from backup system table + * @return get operation + * @throws IOException exception + */ + private Get createGetForIncrBackupTableSet(String backupRoot) throws IOException { + Get get = new Get(rowkey(INCR_BACKUP_SET, backupRoot)); + get.addFamily(BackupSystemTable.META_FAMILY); + get.readVersions(1); + return get; + } + + /** + * Creates Put to store incremental backup table set + * @param tables tables + * @return put operation + */ + private Put createPutForIncrBackupTableSet(Set tables, String backupRoot) { + Put put = new Put(rowkey(INCR_BACKUP_SET, backupRoot)); + for (TableName table : tables) { + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()), + EMPTY_VALUE); + } + return put; + } + + /** + * Creates Delete for incremental backup table set + * @param backupRoot backup root + * @return delete operation + */ + private Delete createDeleteForIncrBackupTableSet(String backupRoot) { + Delete delete = new Delete(rowkey(INCR_BACKUP_SET, backupRoot)); + delete.addFamily(BackupSystemTable.META_FAMILY); + return delete; + } + + /** + * Creates Scan operation to load backup history + * @return scan operation + */ + private Scan createScanForBackupHistory() { + Scan scan = new Scan(); + byte[] startRow = Bytes.toBytes(BACKUP_INFO_PREFIX); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.SESSIONS_FAMILY); + scan.readVersions(1); + return scan; + } + + /** + * Converts cell to backup info instance. + * @param current current cell + * @return backup backup info instance + * @throws IOException exception + */ + private BackupInfo cellToBackupInfo(Cell current) throws IOException { + byte[] data = CellUtil.cloneValue(current); + return BackupInfo.fromByteArray(data); + } + + /** + * Creates Put to write RS last roll log timestamp map + * @param table table + * @param smap map, containing RS:ts + * @return put operation + */ + private Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap, + String backupRoot) { + Put put = new Put(rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot, NULL, table.getNameAsString())); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("log-roll-map"), smap); + return put; + } + + /** + * Creates Scan to load table-> { RS -> ts} map of maps + * @return scan operation + */ + private Scan createScanForReadLogTimestampMap(String backupRoot) { + Scan scan = new Scan(); + byte[] startRow = rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + + return scan; + } + + /** + * Get table name from rowkey + * @param cloneRow rowkey + * @return table name + */ + private String getTableNameForReadLogTimestampMap(byte[] cloneRow) { + String s = Bytes.toString(cloneRow); + int index = s.lastIndexOf(NULL); + return s.substring(index + 1); + } + + /** + * Creates Put to store RS last log result + * @param server server name + * @param timestamp log roll result (timestamp) + * @return put operation + */ + private Put createPutForRegionServerLastLogRollResult(String server, Long timestamp, + String backupRoot) { + Put put = new Put(rowkey(RS_LOG_TS_PREFIX, backupRoot, NULL, server)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("rs-log-ts"), + Bytes.toBytes(timestamp)); + return put; + } + + /** + * Creates Scan operation to load last RS log roll results + * @return scan operation + */ + private Scan createScanForReadRegionServerLastLogRollResult(String backupRoot) { + Scan scan = new Scan(); + byte[] startRow = rowkey(RS_LOG_TS_PREFIX, backupRoot); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + scan.readVersions(1); + + return scan; + } + + /** + * Get server's name from rowkey + * @param row rowkey + * @return server's name + */ + private String getServerNameForReadRegionServerLastLogRollResult(byte[] row) { + String s = Bytes.toString(row); + int index = s.lastIndexOf(NULL); + return s.substring(index + 1); + } + + /* + * Creates Put's for bulk load resulting from running LoadIncrementalHFiles + */ + static List createPutForCommittedBulkload(TableName table, byte[] region, + Map> finalPaths) { + List puts = new ArrayList<>(); + for (Map.Entry> entry : finalPaths.entrySet()) { + for (Path path : entry.getValue()) { + String file = path.toString(); + int lastSlash = file.lastIndexOf("/"); + String filename = file.substring(lastSlash + 1); + Put put = new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, + Bytes.toString(region), BLK_LD_DELIM, filename)); + put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName()); + put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey()); + put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(file)); + put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT); + puts.add(put); + LOG + .debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region)); + } + } + return puts; + } + + public static void snapshot(Connection conn) throws IOException { + try (Admin admin = conn.getAdmin()) { + Configuration conf = conn.getConfiguration(); + admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf)); + } + } + + public static void restoreFromSnapshot(Connection conn) throws IOException { + Configuration conf = conn.getConfiguration(); + LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot"); + try (Admin admin = conn.getAdmin()) { + String snapshotName = BackupSystemTable.getSnapshotName(conf); + if (snapshotExists(admin, snapshotName)) { + admin.disableTable(BackupSystemTable.getTableName(conf)); + admin.restoreSnapshot(snapshotName); + admin.enableTable(BackupSystemTable.getTableName(conf)); + LOG.debug("Done restoring backup system table"); + } else { + // Snapshot does not exists, i.e completeBackup failed after + // deleting backup system table snapshot + // In this case we log WARN and proceed + LOG.warn( + "Could not restore backup system table. Snapshot " + snapshotName + " does not exists."); + } + } + } + + private static boolean snapshotExists(Admin admin, String snapshotName) throws IOException { + List list = admin.listSnapshots(); + for (SnapshotDescription desc : list) { + if (desc.getName().equals(snapshotName)) { + return true; + } + } + return false; + } + + public static boolean snapshotExists(Connection conn) throws IOException { + return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration())); + } + + public static void deleteSnapshot(Connection conn) throws IOException { + Configuration conf = conn.getConfiguration(); + LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system"); + try (Admin admin = conn.getAdmin()) { + String snapshotName = BackupSystemTable.getSnapshotName(conf); + if (snapshotExists(admin, snapshotName)) { + admin.deleteSnapshot(snapshotName); + LOG.debug("Done deleting backup system table snapshot"); + } else { + LOG.error("Snapshot " + snapshotName + " does not exists"); + } + } + } + + /* + * Creates Put's for bulk load resulting from running LoadIncrementalHFiles + */ + static List createPutForPreparedBulkload(TableName table, byte[] region, final byte[] family, + final List> pairs) { + List puts = new ArrayList<>(pairs.size()); + for (Pair pair : pairs) { + Path path = pair.getSecond(); + String file = path.toString(); + int lastSlash = file.lastIndexOf("/"); + String filename = file.substring(lastSlash + 1); + Put put = new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, + Bytes.toString(region), BLK_LD_DELIM, filename)); + put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName()); + put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family); + put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(file)); + put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE); + puts.add(put); + LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region)); + } + return puts; + } + + public static List createDeleteForOrigBulkLoad(List lst) { + List lstDels = new ArrayList<>(lst.size()); + for (TableName table : lst) { + Delete del = new Delete(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM)); + del.addFamily(BackupSystemTable.META_FAMILY); + lstDels.add(del); + } + return lstDels; + } + + private Put createPutForDeleteOperation(String[] backupIdList) { + byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ",")); + Put put = new Put(DELETE_OP_ROW); + put.addColumn(META_FAMILY, FAM_COL, value); + return put; + } + + private Delete createDeleteForBackupDeleteOperation() { + Delete delete = new Delete(DELETE_OP_ROW); + delete.addFamily(META_FAMILY); + return delete; + } + + private Get createGetForDeleteOperation() { + Get get = new Get(DELETE_OP_ROW); + get.addFamily(META_FAMILY); + return get; + } + + public void startDeleteOperation(String[] backupIdList) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList)); + } + Put put = createPutForDeleteOperation(backupIdList); + try (Table table = connection.getTable(tableName)) { + table.put(put); + } + } + + public void finishDeleteOperation() throws IOException { + LOG.trace("Finsih delete operation for backup ids"); + + Delete delete = createDeleteForBackupDeleteOperation(); + try (Table table = connection.getTable(tableName)) { + table.delete(delete); + } + } + + public String[] getListOfBackupIdsFromDeleteOperation() throws IOException { + LOG.trace("Get delete operation for backup ids"); + + Get get = createGetForDeleteOperation(); + try (Table table = connection.getTable(tableName)) { + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + Cell cell = res.listCells().get(0); + byte[] val = CellUtil.cloneValue(cell); + if (val.length == 0) { + return null; + } + return Splitter.on(',').splitToStream(new String(val, StandardCharsets.UTF_8)) + .toArray(String[]::new); + } + } + + private Put createPutForMergeOperation(String[] backupIdList) { + byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ",")); + Put put = new Put(MERGE_OP_ROW); + put.addColumn(META_FAMILY, FAM_COL, value); + return put; + } + + public boolean isMergeInProgress() throws IOException { + Get get = new Get(MERGE_OP_ROW); + try (Table table = connection.getTable(tableName)) { + Result res = table.get(get); + return !res.isEmpty(); + } + } + + private Put createPutForUpdateTablesForMerge(List tables) { + byte[] value = Bytes.toBytes(StringUtils.join(tables, ",")); + Put put = new Put(MERGE_OP_ROW); + put.addColumn(META_FAMILY, PATH_COL, value); + return put; + } + + private Delete createDeleteForBackupMergeOperation() { + Delete delete = new Delete(MERGE_OP_ROW); + delete.addFamily(META_FAMILY); + return delete; + } + + private Get createGetForMergeOperation() { + Get get = new Get(MERGE_OP_ROW); + get.addFamily(META_FAMILY); + return get; + } + + public void startMergeOperation(String[] backupIdList) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList)); + } + Put put = createPutForMergeOperation(backupIdList); + try (Table table = connection.getTable(tableName)) { + table.put(put); + } + } + + public void updateProcessedTablesForMerge(List tables) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Update tables for merge : " + StringUtils.join(tables, ",")); + } + Put put = createPutForUpdateTablesForMerge(tables); + try (Table table = connection.getTable(tableName)) { + table.put(put); + } + } + + public void finishMergeOperation() throws IOException { + LOG.trace("Finish merge operation for backup ids"); + + Delete delete = createDeleteForBackupMergeOperation(); + try (Table table = connection.getTable(tableName)) { + table.delete(delete); + } + } + + public String[] getListOfBackupIdsFromMergeOperation() throws IOException { + LOG.trace("Get backup ids for merge operation"); + + Get get = createGetForMergeOperation(); + try (Table table = connection.getTable(tableName)) { + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + Cell cell = res.listCells().get(0); + byte[] val = CellUtil.cloneValue(cell); + if (val.length == 0) { + return null; + } + return Splitter.on(',').splitToStream(new String(val, StandardCharsets.UTF_8)) + .toArray(String[]::new); + } + } + + static Scan createScanForOrigBulkLoadedFiles(TableName table) { + Scan scan = new Scan(); + byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + scan.readVersions(1); + return scan; + } + + static String getTableNameFromOrigBulkLoadRow(String rowStr) { + // format is bulk : namespace : table : region : file + return Iterators.get(Splitter.onPattern(BLK_LD_DELIM).split(rowStr).iterator(), 1); + } + + static String getRegionNameFromOrigBulkLoadRow(String rowStr) { + // format is bulk : namespace : table : region : file + List parts = Splitter.onPattern(BLK_LD_DELIM).splitToList(rowStr); + Iterator i = parts.iterator(); + int idx = 3; + if (parts.size() == 4) { + // the table is in default namespace + idx = 2; + } + String region = Iterators.get(i, idx); + LOG.debug("bulk row string " + rowStr + " region " + region); + return region; + } + + /* + * Used to query bulk loaded hfiles which have been copied by incremental backup + * @param backupId the backup Id. It can be null when querying for all tables + * @return the Scan object + */ + static Scan createScanForBulkLoadedFiles(String backupId) { + Scan scan = new Scan(); + byte[] startRow = + backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + scan.readVersions(1); + return scan; + } + + static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId, + long ts, int idx) { + Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx)); + put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName()); + put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam); + put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, Bytes.toBytes(p)); + return put; + } + + /** + * Creates Scan operation to load backup set list + * @return scan operation + */ + private Scan createScanForBackupSetList() { + Scan scan = new Scan(); + byte[] startRow = Bytes.toBytes(SET_KEY_PREFIX); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.withStartRow(startRow); + scan.withStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + return scan; + } + + /** + * Creates Get operation to load backup set content + * @return get operation + */ + private Get createGetForBackupSet(String name) { + Get get = new Get(rowkey(SET_KEY_PREFIX, name)); + get.addFamily(BackupSystemTable.META_FAMILY); + return get; + } + + /** + * Creates Delete operation to delete backup set content + * @param name backup set's name + * @return delete operation + */ + private Delete createDeleteForBackupSet(String name) { + Delete del = new Delete(rowkey(SET_KEY_PREFIX, name)); + del.addFamily(BackupSystemTable.META_FAMILY); + return del; + } + + /** + * Creates Put operation to update backup set content + * @param name backup set's name + * @param tables list of tables + * @return put operation + */ + private Put createPutForBackupSet(String name, String[] tables) { + Put put = new Put(rowkey(SET_KEY_PREFIX, name)); + byte[] value = convertToByteArray(tables); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("tables"), value); + return put; + } + + private byte[] convertToByteArray(String[] tables) { + return Bytes.toBytes(StringUtils.join(tables, ",")); + } + + /** + * Converts cell to backup set list. + * @param current current cell + * @return backup set as array of table names + */ + private String[] cellValueToBackupSet(Cell current) { + byte[] data = CellUtil.cloneValue(current); + if (!ArrayUtils.isEmpty(data)) { + return Bytes.toString(data).split(","); + } + return new String[0]; + } + + /** + * Converts cell key to backup set name. + * @param current current cell + * @return backup set name + */ + private String cellKeyToBackupSetName(Cell current) { + byte[] data = CellUtil.cloneRow(current); + return Bytes.toString(data).substring(SET_KEY_PREFIX.length()); + } + + private static byte[] rowkey(String s, String... other) { + StringBuilder sb = new StringBuilder(s); + for (String ss : other) { + sb.append(ss); + } + return Bytes.toBytes(sb.toString()); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/ExclusiveOperationException.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/ExclusiveOperationException.java new file mode 100644 index 000000000000..d5c4ab31c655 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/ExclusiveOperationException.java @@ -0,0 +1,31 @@ +/* + * 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 org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +@SuppressWarnings("serial") +public class ExclusiveOperationException extends IOException { + + public ExclusiveOperationException() { + super(); + } + +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java new file mode 100644 index 000000000000..fee2e825728e --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -0,0 +1,226 @@ +/* + * 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 static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Full table backup implementation + */ +@InterfaceAudience.Private +public class FullTableBackupClient extends TableBackupClient { + private static final Logger LOG = LoggerFactory.getLogger(FullTableBackupClient.class); + + public FullTableBackupClient() { + } + + public FullTableBackupClient(final Connection conn, final String backupId, BackupRequest request) + throws IOException { + super(conn, backupId, request); + } + + /** + * Do snapshot copy. + * @param backupInfo backup info + * @throws Exception exception + */ + protected void snapshotCopy(BackupInfo backupInfo) throws Exception { + LOG.info("Snapshot copy is starting."); + + // set overall backup phase: snapshot_copy + backupInfo.setPhase(BackupPhase.SNAPSHOTCOPY); + + // call ExportSnapshot to copy files based on hbase snapshot for backup + // ExportSnapshot only support single snapshot export, need loop for multiple tables case + BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf); + + // number of snapshots matches number of tables + float numOfSnapshots = backupInfo.getSnapshotNames().size(); + + LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied."); + + for (TableName table : backupInfo.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; + ArrayList argsList = new ArrayList<>(); + argsList.add("-snapshot"); + argsList.add(backupInfo.getSnapshotName(table)); + argsList.add("-copy-to"); + argsList.add(backupInfo.getTableBackupDir(table)); + if (backupInfo.getBandwidth() > -1) { + argsList.add("-bandwidth"); + argsList.add(String.valueOf(backupInfo.getBandwidth())); + } + if (backupInfo.getWorkers() > -1) { + argsList.add("-mappers"); + argsList.add(String.valueOf(backupInfo.getWorkers())); + } + + String[] args = argsList.toArray(new String[0]); + + String jobname = "Full-Backup_" + backupInfo.getBackupId() + "_" + table.getNameAsString(); + if (LOG.isDebugEnabled()) { + LOG.debug("Setting snapshot copy job name to : " + jobname); + } + conf.set(JOB_NAME_CONF_KEY, jobname); + + LOG.debug("Copy snapshot " + args[1] + " to " + args[3]); + res = copyService.copy(backupInfo, backupManager, conf, BackupType.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); + } + + conf.unset(JOB_NAME_CONF_KEY); + LOG.info("Snapshot copy " + args[1] + " finished."); + } + } + + /** + * Backup request execution. + * @throws IOException if the execution of the backup fails + */ + @Override + public void execute() throws IOException { + try (Admin admin = conn.getAdmin()) { + // Begin BACKUP + beginBackup(backupManager, backupInfo); + String savedStartCode; + boolean firstBackup; + // do snapshot for full table backup + + savedStartCode = backupManager.readBackupStartCode(); + firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; + if (firstBackup) { + // This is our first backup. Let's put some marker to system table 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 ..."); + + Map props = new HashMap<>(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + + // SNAPSHOT_TABLES: + backupInfo.setPhase(BackupPhase.SNAPSHOT); + for (TableName tableName : tableList) { + String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString(); + + snapshotTable(admin, tableName, snapshotName); + backupInfo.setSnapshotName(tableName, snapshotName); + } + + // SNAPSHOT_COPY: + // do snapshot copy + LOG.debug("snapshot copy for " + backupId); + snapshotCopy(backupInfo); + // Updates incremental backup table set + backupManager.addIncrementalBackupTableSet(backupInfo.getTables()); + + // 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 + backupInfo.setState(BackupState.COMPLETE); + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + Map> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + backupInfo.setTableSetTimestampMap(newTableSetTimestampMap); + Long newStartCode = + BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf); + } catch (Exception e) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", + BackupType.FULL, conf); + throw new IOException(e); + } + } + + protected void snapshotTable(Admin admin, TableName tableName, String snapshotName) + throws IOException { + int maxAttempts = conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS); + int pause = conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS); + int attempts = 0; + + while (attempts++ < maxAttempts) { + try { + admin.snapshot(snapshotName, tableName); + return; + } catch (IOException ee) { + LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName + + ", sleeping for " + pause + "ms", ee); + if (attempts < maxAttempts) { + try { + Thread.sleep(pause); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + } + throw new IOException("Failed to snapshot table " + tableName); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java new file mode 100644 index 000000000000..c92c0747e83c --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +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.fs.PathFilter; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * After a full backup was created, the incremental backup will only store the changes made after + * the last full or incremental backup. Creating the backup copies the logfiles in .logs and + * .oldlogs since the last backup timestamp. + */ +@InterfaceAudience.Private +public class IncrementalBackupManager extends BackupManager { + public static final Logger LOG = LoggerFactory.getLogger(IncrementalBackupManager.class); + + public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException { + super(conn, conf); + } + + /** + * Obtain the list of logs that need to be copied out for this incremental backup. The list is set + * in BackupInfo. + * @return The new HashMap of RS log time stamps after the log roll for this incremental backup. + * @throws IOException exception + */ + public Map getIncrBackupLogFileMap() throws IOException { + List logList; + Map newTimestamps; + Map previousTimestampMins; + + String savedStartCode = readBackupStartCode(); + + // key: tableName + // value: + Map> previousTimestampMap = readLogTimestampMap(); + + previousTimestampMins = BackupUtils.getRSLogTimestampMins(previousTimestampMap); + + if (LOG.isDebugEnabled()) { + LOG.debug("StartCode " + savedStartCode + "for backupID " + backupInfo.getBackupId()); + } + // get all new log files from .logs and .oldlogs after last TS and before new timestamp + if ( + savedStartCode == null || previousTimestampMins == null || previousTimestampMins.isEmpty() + ) { + throw new IOException("Cannot read any previous back up timestamps from backup system table. " + + "In order to create an incremental backup, at least one full backup is needed."); + } + + LOG.info("Execute roll log procedure for incremental backup ..."); + HashMap props = new HashMap<>(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + + try (Admin admin = conn.getAdmin()) { + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + } + newTimestamps = readRegionServerLastLogRollResult(); + + logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode); + logList = excludeProcV2WALs(logList); + backupInfo.setIncrBackupFileList(logList); + + return newTimestamps; + } + + private List excludeProcV2WALs(List logList) { + List list = new ArrayList<>(); + for (int i = 0; i < logList.size(); i++) { + Path p = new Path(logList.get(i)); + String name = p.getName(); + + if (name.startsWith(WALProcedureStore.LOG_PREFIX)) { + continue; + } + + list.add(logList.get(i)); + } + return list; + } + + /** + * For each region server: get all log files newer than the last timestamps but not newer than the + * newest timestamps. + * @param olderTimestamps the timestamp for each region server of the last backup. + * @param newestTimestamps the timestamp for each region server that the backup should lead to. + * @param conf the Hadoop and Hbase configuration + * @param savedStartCode the startcode (timestamp) of last successful backup. + * @return a list of log files to be backed up + * @throws IOException exception + */ + private List getLogFilesForNewBackup(Map olderTimestamps, + Map newestTimestamps, Configuration conf, String savedStartCode) + throws IOException { + LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps + + "\n newestTimestamps: " + newestTimestamps); + + Path walRootDir = CommonFSUtils.getWALRootDir(conf); + Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); + FileSystem fs = walRootDir.getFileSystem(conf); + NewestLogFilter pathFilter = new NewestLogFilter(); + + List resultLogFiles = new ArrayList<>(); + List newestLogs = new ArrayList<>(); + + /* + * The old region servers and timestamps info we kept in backup system table may be out of sync + * if new region server is added or existing one lost. We'll deal with it here when processing + * the logs. If data in backup system table has more hosts, just ignore it. If the .logs + * directory includes more hosts, the additional hosts will not have old timestamps to compare + * with. We'll just use all the logs in that directory. We always write up-to-date region server + * and timestamp info to backup system table at the end of successful backup. + */ + FileStatus[] rss; + Path p; + String host; + Long oldTimeStamp; + String currentLogFile; + long currentLogTS; + + // Get the files in .logs. + rss = fs.listStatus(logDir); + for (FileStatus rs : rss) { + p = rs.getPath(); + host = BackupUtils.parseHostNameFromLogFile(p); + if (host == null) { + continue; + } + FileStatus[] logs; + oldTimeStamp = olderTimestamps.get(host); + // It is possible that there is no old timestamp in backup system table for this host if + // this region server is newly added after our last backup. + if (oldTimeStamp == null) { + logs = fs.listStatus(p); + } else { + pathFilter.setLastBackupTS(oldTimeStamp); + logs = fs.listStatus(p, pathFilter); + } + for (FileStatus log : logs) { + LOG.debug("currentLogFile: " + log.getPath().toString()); + if (AbstractFSWALProvider.isMetaFile(log.getPath())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip hbase:meta log file: " + log.getPath().getName()); + } + continue; + } + currentLogFile = log.getPath().toString(); + resultLogFiles.add(currentLogFile); + currentLogTS = BackupUtils.getCreationTime(log.getPath()); + + // If newestTimestamps.get(host) is null, means that + // either RS (host) has been restarted recently with different port number + // or RS is down (was decommisioned). In any case, we treat this + // log file as eligible for inclusion into incremental backup log list + Long ts = newestTimestamps.get(host); + if (ts == null) { + LOG.warn("ORPHAN log found: " + log + " host=" + host); + LOG.debug("Known hosts (from newestTimestamps):"); + for (String s : newestTimestamps.keySet()) { + LOG.debug(s); + } + } + if (ts == null || currentLogTS > ts) { + newestLogs.add(currentLogFile); + } + } + } + + // Include the .oldlogs files too. + FileStatus[] oldlogs = fs.listStatus(oldLogDir); + for (FileStatus oldlog : oldlogs) { + p = oldlog.getPath(); + currentLogFile = p.toString(); + if (AbstractFSWALProvider.isMetaFile(p)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip .meta log file: " + currentLogFile); + } + continue; + } + host = BackupUtils.parseHostFromOldLog(p); + if (host == null) { + continue; + } + currentLogTS = BackupUtils.getCreationTime(p); + oldTimeStamp = olderTimestamps.get(host); + /* + * It is possible that there is no old timestamp in backup system table for this host. At the + * time of our last backup operation, this rs did not exist. The reason can be one of the two: + * 1. The rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after + * our last backup. + */ + if (oldTimeStamp == null) { + if (currentLogTS < Long.parseLong(savedStartCode)) { + // This log file is really old, its region server was before our last backup. + continue; + } else { + resultLogFiles.add(currentLogFile); + } + } else if (currentLogTS > oldTimeStamp) { + resultLogFiles.add(currentLogFile); + } + + // It is possible that a host in .oldlogs is an obsolete region server + // so newestTimestamps.get(host) here can be null. + // Even if these logs belong to a obsolete region server, we still need + // to include they to avoid loss of edits for backup. + Long newTimestamp = newestTimestamps.get(host); + if (newTimestamp == null || currentLogTS > newTimestamp) { + newestLogs.add(currentLogFile); + } + } + // remove newest log per host because they are still in use + resultLogFiles.removeAll(newestLogs); + return resultLogFiles; + } + + static class NewestLogFilter implements PathFilter { + private Long lastBackupTS = 0L; + + public NewestLogFilter() { + } + + protected void setLastBackupTS(Long ts) { + this.lastBackupTS = ts; + } + + @Override + public boolean accept(Path path) { + // skip meta table log -- ts.meta file + if (AbstractFSWALProvider.isMetaFile(path)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip .meta log file: " + path.getName()); + } + return false; + } + long timestamp; + try { + timestamp = BackupUtils.getCreationTime(path); + return timestamp > lastBackupTS; + } catch (Exception e) { + LOG.warn("Cannot read timestamp of log file " + path); + return false; + } + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java new file mode 100644 index 000000000000..87cb722b89e3 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -0,0 +1,436 @@ +/* + * 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 static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.util.Tool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Incremental backup implementation. See the {@link #execute() execute} method. + */ +@InterfaceAudience.Private +public class IncrementalTableBackupClient extends TableBackupClient { + private static final Logger LOG = LoggerFactory.getLogger(IncrementalTableBackupClient.class); + + protected IncrementalTableBackupClient() { + } + + public IncrementalTableBackupClient(final Connection conn, final String backupId, + BackupRequest request) throws IOException { + super(conn, backupId, request); + } + + protected List filterMissingFiles(List incrBackupFileList) throws IOException { + List list = new ArrayList<>(); + for (String file : incrBackupFileList) { + Path p = new Path(file); + if (fs.exists(p) || isActiveWalPath(p)) { + list.add(file); + } else { + LOG.warn("Can't find file: " + file); + } + } + return list; + } + + /** + * Check if a given path is belongs to active WAL directory + * @param p path + * @return true, if yes + */ + protected boolean isActiveWalPath(Path p) { + return !AbstractFSWALProvider.isArchivedLogFile(p); + } + + protected static int getIndex(TableName tbl, List sTableList) { + if (sTableList == null) { + return 0; + } + + for (int i = 0; i < sTableList.size(); i++) { + if (tbl.equals(sTableList.get(i))) { + return i; + } + } + return -1; + } + + /* + * Reads bulk load records from backup table, iterates through the records and forms the paths for + * bulk loaded hfiles. Copies the bulk loaded hfiles to backup destination + * @param sTableList list of tables to be backed up + * @return map of table to List of files + */ + @SuppressWarnings("unchecked") + protected Map>[] handleBulkLoad(List sTableList) + throws IOException { + Map>[] mapForSrc = new Map[sTableList.size()]; + List activeFiles = new ArrayList<>(); + List archiveFiles = new ArrayList<>(); + Pair>>>>, List> pair = + backupManager.readBulkloadRows(sTableList); + Map>>>> map = pair.getFirst(); + FileSystem tgtFs; + try { + tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf); + } catch (URISyntaxException use) { + throw new IOException("Unable to get FileSystem", use); + } + Path rootdir = CommonFSUtils.getRootDir(conf); + Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId); + + for (Map.Entry>>>> tblEntry : map + .entrySet()) { + TableName srcTable = tblEntry.getKey(); + + int srcIdx = getIndex(srcTable, sTableList); + if (srcIdx < 0) { + LOG.warn("Couldn't find " + srcTable + " in source table List"); + continue; + } + if (mapForSrc[srcIdx] == null) { + mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR); + } + Path tblDir = CommonFSUtils.getTableDir(rootdir, srcTable); + Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()), + srcTable.getQualifierAsString()); + for (Map.Entry>>> regionEntry : tblEntry + .getValue().entrySet()) { + String regionName = regionEntry.getKey(); + Path regionDir = new Path(tblDir, regionName); + // map from family to List of hfiles + for (Map.Entry>> famEntry : regionEntry.getValue() + .entrySet()) { + String fam = famEntry.getKey(); + Path famDir = new Path(regionDir, fam); + List files; + if (!mapForSrc[srcIdx].containsKey(Bytes.toBytes(fam))) { + files = new ArrayList<>(); + mapForSrc[srcIdx].put(Bytes.toBytes(fam), files); + } else { + files = mapForSrc[srcIdx].get(Bytes.toBytes(fam)); + } + Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam); + String tblName = srcTable.getQualifierAsString(); + Path tgtFam = new Path(new Path(tgtTable, regionName), fam); + if (!tgtFs.mkdirs(tgtFam)) { + throw new IOException("couldn't create " + tgtFam); + } + for (Pair fileWithState : famEntry.getValue()) { + String file = fileWithState.getFirst(); + int idx = file.lastIndexOf("/"); + String filename = file; + if (idx > 0) { + filename = file.substring(idx + 1); + } + Path p = new Path(famDir, filename); + Path tgt = new Path(tgtFam, filename); + Path archive = new Path(archiveDir, filename); + if (fs.exists(p)) { + if (LOG.isTraceEnabled()) { + LOG.trace("found bulk hfile " + file + " in " + famDir + " for " + tblName); + } + if (LOG.isTraceEnabled()) { + LOG.trace("copying " + p + " to " + tgt); + } + activeFiles.add(p.toString()); + } else if (fs.exists(archive)) { + LOG.debug("copying archive " + archive + " to " + tgt); + archiveFiles.add(archive.toString()); + } + files.add(tgt); + } + } + } + } + + copyBulkLoadedFiles(activeFiles, archiveFiles); + backupManager.deleteBulkLoadedRows(pair.getSecond()); + return mapForSrc; + } + + private void copyBulkLoadedFiles(List activeFiles, List archiveFiles) + throws IOException { + try { + // Enable special mode of BackupDistCp + conf.setInt(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY, 5); + // Copy active files + String tgtDest = backupInfo.getBackupRootDir() + Path.SEPARATOR + backupInfo.getBackupId(); + int attempt = 1; + while (activeFiles.size() > 0) { + LOG.info("Copy " + activeFiles.size() + " active bulk loaded files. Attempt =" + attempt++); + String[] toCopy = new String[activeFiles.size()]; + activeFiles.toArray(toCopy); + // Active file can be archived during copy operation, + // we need to handle this properly + try { + incrementalCopyHFiles(toCopy, tgtDest); + break; + } catch (IOException e) { + // Check if some files got archived + // Update active and archived lists + // When file is being moved from active to archive + // directory, the number of active files decreases + int numOfActive = activeFiles.size(); + updateFileLists(activeFiles, archiveFiles); + if (activeFiles.size() < numOfActive) { + continue; + } + // if not - throw exception + throw e; + } + } + // If incremental copy will fail for archived files + // we will have partially loaded files in backup destination (only files from active data + // directory). It is OK, because the backup will marked as FAILED and data will be cleaned up + if (archiveFiles.size() > 0) { + String[] toCopy = new String[archiveFiles.size()]; + archiveFiles.toArray(toCopy); + incrementalCopyHFiles(toCopy, tgtDest); + } + } finally { + // Disable special mode of BackupDistCp + conf.unset(MapReduceBackupCopyJob.NUMBER_OF_LEVELS_TO_PRESERVE_KEY); + } + } + + private void updateFileLists(List activeFiles, List archiveFiles) + throws IOException { + List newlyArchived = new ArrayList<>(); + + for (String spath : activeFiles) { + if (!fs.exists(new Path(spath))) { + newlyArchived.add(spath); + } + } + + if (newlyArchived.size() > 0) { + activeFiles.removeAll(newlyArchived); + archiveFiles.addAll(newlyArchived); + } + + LOG.debug(newlyArchived.size() + " files have been archived."); + } + + @Override + public void execute() throws IOException { + try { + // case PREPARE_INCREMENTAL: + beginBackup(backupManager, backupInfo); + backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); + LOG.debug("For incremental backup, current table set is " + + backupManager.getIncrementalBackupTableSet()); + newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap(); + } catch (Exception e) { + // fail the overall backup and return + failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", + BackupType.INCREMENTAL, conf); + throw new IOException(e); + } + + // case INCREMENTAL_COPY: + try { + // copy out the table and region info files for each table + BackupUtils.copyTableRegionInfo(conn, backupInfo, conf); + // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT + convertWALsToHFiles(); + incrementalCopyHFiles(new String[] { getBulkOutputDir().toString() }, + backupInfo.getBackupRootDir()); + } catch (Exception e) { + String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId; + // fail the overall backup and return + failBackup(conn, backupInfo, backupManager, e, msg, BackupType.INCREMENTAL, conf); + throw new IOException(e); + } + // 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 + try { + // Set the previousTimestampMap which is before this current log roll to the manifest. + Map> previousTimestampMap = backupManager.readLogTimestampMap(); + backupInfo.setIncrTimestampMap(previousTimestampMap); + + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + Map> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + backupInfo.setTableSetTimestampMap(newTableSetTimestampMap); + Long newStartCode = + BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + + handleBulkLoad(backupInfo.getTableNames()); + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf); + + } catch (IOException e) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", + BackupType.INCREMENTAL, conf); + throw new IOException(e); + } + } + + protected void incrementalCopyHFiles(String[] files, String backupDest) throws IOException { + try { + LOG.debug("Incremental copy HFiles is starting. dest=" + backupDest); + // set overall backup phase: incremental_copy + backupInfo.setPhase(BackupPhase.INCREMENTAL_COPY); + // get incremental backup file list and prepare parms for DistCp + String[] strArr = new String[files.length + 1]; + System.arraycopy(files, 0, strArr, 0, files.length); + strArr[strArr.length - 1] = backupDest; + + String jobname = "Incremental_Backup-HFileCopy-" + backupInfo.getBackupId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Setting incremental copy HFiles job name to : " + jobname); + } + conf.set(JOB_NAME_CONF_KEY, jobname); + + BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf); + int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr); + if (res != 0) { + LOG.error("Copy incremental HFile files failed with return code: " + res + "."); + throw new IOException( + "Failed copy from " + StringUtils.join(files, ',') + " to " + backupDest); + } + LOG.debug("Incremental copy HFiles from " + StringUtils.join(files, ',') + " to " + backupDest + + " finished."); + } finally { + deleteBulkLoadDirectory(); + } + } + + protected void deleteBulkLoadDirectory() throws IOException { + // delete original bulk load directory on method exit + Path path = getBulkOutputDir(); + FileSystem fs = FileSystem.get(path.toUri(), conf); + boolean result = fs.delete(path, true); + if (!result) { + LOG.warn("Could not delete " + path); + } + } + + protected void convertWALsToHFiles() throws IOException { + // get incremental backup file list and prepare parameters for DistCp + List incrBackupFileList = backupInfo.getIncrBackupFileList(); + // Get list of tables in incremental backup set + Set tableSet = backupManager.getIncrementalBackupTableSet(); + // filter missing files out (they have been copied by previous backups) + incrBackupFileList = filterMissingFiles(incrBackupFileList); + List tableList = new ArrayList(); + for (TableName table : tableSet) { + // Check if table exists + if (tableExists(table, conn)) { + tableList.add(table.getNameAsString()); + } else { + LOG.warn("Table " + table + " does not exists. Skipping in WAL converter"); + } + } + walToHFiles(incrBackupFileList, tableList); + + } + + protected boolean tableExists(TableName table, Connection conn) throws IOException { + try (Admin admin = conn.getAdmin()) { + return admin.tableExists(table); + } + } + + protected void walToHFiles(List dirPaths, List tableList) throws IOException { + Tool player = new WALPlayer(); + + // Player reads all files in arbitrary directory structure and creates + // a Map task for each file. We use ';' as separator + // because WAL file names contains ',' + String dirs = StringUtils.join(dirPaths, ';'); + String jobname = "Incremental_Backup-" + backupId; + + Path bulkOutputPath = getBulkOutputDir(); + conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString()); + conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";"); + conf.setBoolean(HFileOutputFormat2.TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY, true); + conf.setBoolean(WALPlayer.MULTI_TABLES_SUPPORT, true); + conf.set(JOB_NAME_CONF_KEY, jobname); + String[] playerArgs = { dirs, StringUtils.join(tableList, ",") }; + + try { + player.setConf(conf); + int result = player.run(playerArgs); + if (result != 0) { + throw new IOException("WAL Player failed"); + } + conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY); + conf.unset(JOB_NAME_CONF_KEY); + } catch (IOException e) { + throw e; + } catch (Exception ee) { + throw new IOException("Can not convert from directory " + dirs + + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee); + } + } + + protected Path getBulkOutputDirForTable(TableName table) { + Path tablePath = getBulkOutputDir(); + tablePath = new Path(tablePath, table.getNamespaceAsString()); + tablePath = new Path(tablePath, table.getQualifierAsString()); + return new Path(tablePath, "data"); + } + + protected Path getBulkOutputDir() { + String backupId = backupInfo.getBackupId(); + Path path = new Path(backupInfo.getBackupRootDir()); + path = new Path(path, ".tmp"); + path = new Path(path, backupId); + return path; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java new file mode 100644 index 000000000000..9ec2442a3d93 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -0,0 +1,258 @@ +/* + * 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 static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.TreeSet; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.util.RestoreTool; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Restore table implementation + */ +@InterfaceAudience.Private +public class RestoreTablesClient { + private static final Logger LOG = LoggerFactory.getLogger(RestoreTablesClient.class); + + private Configuration conf; + private Connection conn; + private String backupId; + private TableName[] sTableArray; + private TableName[] tTableArray; + private String targetRootDir; + private boolean isOverwrite; + + public RestoreTablesClient(Connection conn, RestoreRequest request) { + this.targetRootDir = request.getBackupRootDir(); + this.backupId = request.getBackupId(); + this.sTableArray = request.getFromTables(); + this.tTableArray = request.getToTables(); + if (tTableArray == null || tTableArray.length == 0) { + this.tTableArray = sTableArray; + } + this.isOverwrite = request.isOverwrite(); + this.conn = conn; + this.conf = conn.getConfiguration(); + } + + /** + * Validate target tables. + * @param tTableArray target tables + * @param isOverwrite overwrite existing table + * @throws IOException exception + */ + private void checkTargetTables(TableName[] tTableArray, boolean isOverwrite) throws IOException { + ArrayList existTableList = new ArrayList<>(); + ArrayList disabledTableList = new ArrayList<>(); + + // check if the tables already exist + try (Admin admin = conn.getAdmin()) { + for (TableName tableName : tTableArray) { + if (admin.tableExists(tableName)) { + existTableList.add(tableName); + if (admin.isTableDisabled(tableName)) { + disabledTableList.add(tableName); + } + } else { + LOG.info("HBase table " + tableName + + " does not exist. It will be created during restore process"); + } + } + } + + if (existTableList.size() > 0) { + if (!isOverwrite) { + LOG.error("Existing table (" + existTableList + ") found in the restore target, please add " + + "\"-o\" as overwrite option in the command if you mean" + + " to restore to these existing tables"); + throw new IOException( + "Existing table found in target while no \"-o\" " + "as overwrite option found"); + } else { + if (disabledTableList.size() > 0) { + LOG.error("Found offline table in the restore target, " + + "please enable them before restore with \"-overwrite\" option"); + LOG.info("Offline table list in restore target: " + disabledTableList); + throw new IOException( + "Found offline table in the target when restore with \"-overwrite\" option"); + } + } + } + } + + /** + * Restore operation handle each backupImage in array. + * @param images array BackupImage + * @param sTable table to be restored + * @param tTable table to be restored to + * @param truncateIfExists truncate table + * @throws IOException exception + */ + + private void restoreImages(BackupImage[] images, TableName sTable, TableName tTable, + boolean truncateIfExists) throws IOException { + // First image MUST be image of a FULL backup + BackupImage image = images[0]; + String rootDir = image.getRootDir(); + String backupId = image.getBackupId(); + Path backupRoot = new Path(rootDir); + RestoreTool restoreTool = new RestoreTool(conf, backupRoot, backupId); + Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId); + String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId(); + // We need hFS only for full restore (see the code) + BackupManifest manifest = HBackupFileSystem.getManifest(conf, backupRoot, backupId); + if (manifest.getType() == BackupType.FULL) { + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full" + " backup image " + + tableBackupPath.toString()); + conf.set(JOB_NAME_CONF_KEY, "Full_Restore-" + backupId + "-" + tTable); + restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists, + lastIncrBackupId); + conf.unset(JOB_NAME_CONF_KEY); + } else { // incremental Backup + throw new IOException("Unexpected backup type " + image.getType()); + } + + if (images.length == 1) { + // full backup restore done + return; + } + + List dirList = new ArrayList<>(); + // add full backup path + // full backup path comes first + for (int i = 1; i < images.length; i++) { + BackupImage im = images[i]; + String fileBackupDir = + HBackupFileSystem.getTableBackupDir(im.getRootDir(), im.getBackupId(), sTable); + List list = getFilesRecursively(fileBackupDir); + dirList.addAll(list); + + } + + if (dirList.isEmpty()) { + LOG.warn("Nothing has changed, so there is no need to restore '" + sTable + "'"); + return; + } + + String dirs = StringUtils.join(dirList, ","); + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from log dirs: " + dirs); + Path[] paths = new Path[dirList.size()]; + dirList.toArray(paths); + conf.set(JOB_NAME_CONF_KEY, "Incremental_Restore-" + backupId + "-" + tTable); + restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable }, + new TableName[] { tTable }, lastIncrBackupId); + LOG.info(sTable + " has been successfully restored to " + tTable); + } + + private List getFilesRecursively(String fileBackupDir) + throws IllegalArgumentException, IOException { + FileSystem fs = FileSystem.get(new Path(fileBackupDir).toUri(), new Configuration()); + List list = new ArrayList<>(); + RemoteIterator it = fs.listFiles(new Path(fileBackupDir), true); + while (it.hasNext()) { + Path p = it.next().getPath(); + if (HFile.isHFileFormat(fs, p)) { + list.add(p); + } + } + return list; + } + + /** + * Restore operation. Stage 2: resolved Backup Image dependency + * @param backupManifestMap : tableName, Manifest + * @param sTableArray The array of tables to be restored + * @param tTableArray The array of mapping tables to restore to + * @throws IOException exception + */ + private void restore(HashMap backupManifestMap, + TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException { + TreeSet restoreImageSet = new TreeSet<>(); + + for (int i = 0; i < sTableArray.length; i++) { + TableName table = sTableArray[i]; + + BackupManifest manifest = backupManifestMap.get(table); + // Get the image list of this backup for restore in time order from old + // to new. + List list = new ArrayList<>(); + list.add(manifest.getBackupImage()); + TreeSet set = new TreeSet<>(list); + List depList = manifest.getDependentListByTable(table); + set.addAll(depList); + BackupImage[] arr = new BackupImage[set.size()]; + set.toArray(arr); + restoreImages(arr, table, tTableArray[i], isOverwrite); + restoreImageSet.addAll(list); + if (restoreImageSet != null && !restoreImageSet.isEmpty()) { + LOG.info("Restore includes the following image(s):"); + for (BackupImage image : restoreImageSet) { + LOG.info("Backup: " + image.getBackupId() + " " + + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table)); + } + } + } + LOG.debug("restoreStage finished"); + } + + static long getTsFromBackupId(String backupId) { + if (backupId == null) { + return 0; + } + return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1)); + } + + static boolean withinRange(long a, long lower, long upper) { + return a >= lower && a <= upper; + } + + public void execute() throws IOException { + // case VALIDATION: + // check the target tables + checkTargetTables(tTableArray, isOverwrite); + + // case RESTORE_IMAGES: + HashMap backupManifestMap = new HashMap<>(); + // check and load backup image manifest for the tables + Path rootPath = new Path(targetRootDir); + HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath, + backupId); + + restore(backupManifestMap, sTableArray, tTableArray, isOverwrite); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java new file mode 100644 index 000000000000..2bb2c13e4dd8 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java @@ -0,0 +1,423 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class for backup operation. Concrete implementation for full and incremental backup are + * delegated to corresponding sub-classes: {@link FullTableBackupClient} and + * {@link IncrementalTableBackupClient} + */ +@InterfaceAudience.Private +public abstract class TableBackupClient { + + public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class"; + + public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage"; + + private static final Logger LOG = LoggerFactory.getLogger(TableBackupClient.class); + + protected Configuration conf; + protected Connection conn; + protected String backupId; + protected List tableList; + protected Map newTimestamps = null; + + protected BackupManager backupManager; + protected BackupInfo backupInfo; + protected FileSystem fs; + + public TableBackupClient() { + } + + public TableBackupClient(final Connection conn, final String backupId, BackupRequest request) + throws IOException { + init(conn, backupId, request); + } + + public void init(final Connection conn, final String backupId, BackupRequest request) + throws IOException { + if (request.getBackupType() == BackupType.FULL) { + backupManager = new BackupManager(conn, conn.getConfiguration()); + } else { + backupManager = new IncrementalBackupManager(conn, conn.getConfiguration()); + } + this.backupId = backupId; + this.tableList = request.getTableList(); + this.conn = conn; + this.conf = conn.getConfiguration(); + this.fs = CommonFSUtils.getCurrentFileSystem(conf); + backupInfo = backupManager.createBackupInfo(backupId, request.getBackupType(), tableList, + request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth()); + if (tableList == null || tableList.isEmpty()) { + this.tableList = new ArrayList<>(backupInfo.getTables()); + } + // Start new session + backupManager.startBackupSession(); + } + + /** + * Begin the overall backup. + * @param backupInfo backup info + * @throws IOException exception + */ + protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) + throws IOException { + + BackupSystemTable.snapshot(conn); + backupManager.setBackupInfo(backupInfo); + // set the start timestamp of the overall backup + long startTs = EnvironmentEdgeManager.currentTime(); + backupInfo.setStartTs(startTs); + // set overall backup status: ongoing + backupInfo.setState(BackupState.RUNNING); + backupInfo.setPhase(BackupPhase.REQUEST); + LOG.info("Backup " + backupInfo.getBackupId() + " started at " + startTs + "."); + + backupManager.updateBackupInfo(backupInfo); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup session " + backupInfo.getBackupId() + " has been started."); + } + } + + protected 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 backupInfo backup info + * @throws IOException exception + */ + protected static void deleteSnapshots(final Connection conn, BackupInfo backupInfo, + Configuration conf) throws IOException { + LOG.debug("Trying to delete snapshot for full backup."); + for (String snapshotName : backupInfo.getSnapshotNames()) { + if (snapshotName == null) { + continue; + } + LOG.debug("Trying to delete snapshot: " + snapshotName); + + try (Admin admin = conn.getAdmin()) { + admin.deleteSnapshot(snapshotName); + } + LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupInfo.getBackupId() + + " succeeded."); + } + } + + /** + * Clean up directories with prefix "exportSnapshot-", which are generated when exporting + * snapshots. + * @throws IOException exception + */ + protected static void cleanupExportSnapshotLog(Configuration conf) throws IOException { + FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf); + Path stagingDir = new Path( + conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory().toString())); + FileStatus[] files = CommonFSUtils.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 (CommonFSUtils.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. + */ + protected static void cleanupTargetDir(BackupInfo backupInfo, 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: " + backupInfo.getPhase()); + if ( + backupInfo.getPhase().equals(BackupPhase.SNAPSHOTCOPY) + || backupInfo.getPhase().equals(BackupPhase.INCREMENTAL_COPY) + || backupInfo.getPhase().equals(BackupPhase.STORE_MANIFEST) + ) { + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + // now treat one backup as a transaction, clean up data that has been partially copied at + // table level + for (TableName table : backupInfo.getTables()) { + Path targetDirPath = new Path(HBackupFileSystem + .getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.debug( + "Cleaning up uncompleted backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.debug("No data has been copied to " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = CommonFSUtils.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 " + backupInfo.getBackupId() + " at " + + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Fail the overall backup. + * @param backupInfo backup info + * @param e exception + * @throws IOException exception + */ + protected void failBackup(Connection conn, BackupInfo backupInfo, BackupManager backupManager, + Exception e, String msg, BackupType type, Configuration conf) throws IOException { + try { + LOG.error(msg + getMessage(e), e); + // If this is a cancel exception, then we've already cleaned. + // set the failure timestamp of the overall backup + backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); + // set failure message + backupInfo.setFailedMsg(e.getMessage()); + // set overall backup status: failed + backupInfo.setState(BackupState.FAILED); + // compose the backup failed data + String backupFailedData = "BackupId=" + backupInfo.getBackupId() + ",startts=" + + backupInfo.getStartTs() + ",failedts=" + backupInfo.getCompleteTs() + ",failedphase=" + + backupInfo.getPhase() + ",failedmessage=" + backupInfo.getFailedMsg(); + LOG.error(backupFailedData); + cleanupAndRestoreBackupSystem(conn, backupInfo, conf); + // If backup session is updated to FAILED state - means we + // processed recovery already. + backupManager.updateBackupInfo(backupInfo); + backupManager.finishBackupSession(); + LOG.error("Backup " + backupInfo.getBackupId() + " failed."); + } catch (IOException ee) { + LOG.error("Please run backup repair tool manually to restore backup system integrity"); + throw ee; + } + } + + public static void cleanupAndRestoreBackupSystem(Connection conn, BackupInfo backupInfo, + Configuration conf) throws IOException { + BackupType type = backupInfo.getType(); + // 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) { + deleteSnapshots(conn, backupInfo, conf); + cleanupExportSnapshotLog(conf); + } + BackupSystemTable.restoreFromSnapshot(conn); + BackupSystemTable.deleteSnapshot(conn); + // 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(backupInfo, conf); + } + + /** + * Add manifest for the current backup. The manifest is stored within the table backup directory. + * @param backupInfo The current backup info + * @throws IOException exception + */ + protected void addManifest(BackupInfo backupInfo, BackupManager backupManager, BackupType type, + Configuration conf) throws IOException { + // set the overall backup phase : store manifest + backupInfo.setPhase(BackupPhase.STORE_MANIFEST); + + BackupManifest manifest; + + // Since we have each table's backup in its own directory structure, + // we'll store its manifest with the table directory. + for (TableName table : backupInfo.getTables()) { + manifest = new BackupManifest(backupInfo, table); + ArrayList ancestors = backupManager.getAncestors(backupInfo, 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. + Map> tableTimestampMap = new HashMap<>(); + tableTimestampMap.put(table, backupInfo.getIncrTimestampMap().get(table)); + manifest.setIncrTimestampMap(tableTimestampMap); + ArrayList ancestorss = backupManager.getAncestors(backupInfo); + for (BackupImage image : ancestorss) { + manifest.addDependentImage(image); + } + } + manifest.store(conf); + } + + // For incremental backup, we store a overall manifest in + // /WALs/ + // This is used when created the next incremental backup + if (type == BackupType.INCREMENTAL) { + manifest = new BackupManifest(backupInfo); + // set the table region server start and end timestamps for incremental backup + manifest.setIncrTimestampMap(backupInfo.getIncrTimestampMap()); + ArrayList ancestors = backupManager.getAncestors(backupInfo); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + manifest.store(conf); + } + } + + /** + * Get backup request meta data dir as string. + * @param backupInfo backup info + * @return meta data dir + */ + protected String obtainBackupMetaDataStr(BackupInfo backupInfo) { + StringBuilder sb = new StringBuilder(); + sb.append("type=" + backupInfo.getType() + ",tablelist="); + for (TableName table : backupInfo.getTables()) { + sb.append(table + ";"); + } + if (sb.lastIndexOf(";") > 0) { + sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); + } + sb.append(",targetRootDir=" + backupInfo.getBackupRootDir()); + + return sb.toString(); + } + + /** + * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying + * hlogs. + * @throws IOException exception + */ + protected void cleanupDistCpLog(BackupInfo backupInfo, Configuration conf) throws IOException { + Path rootPath = new Path(backupInfo.getHLogTargetDir()).getParent(); + FileStatus[] files = CommonFSUtils.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()); + CommonFSUtils.delete(fs, file.getPath(), true); + } + } + } + + /** + * Complete the overall backup. + * @param backupInfo backup info + * @throws IOException exception + */ + protected void completeBackup(final Connection conn, BackupInfo backupInfo, + BackupManager backupManager, BackupType type, Configuration conf) throws IOException { + // set the complete timestamp of the overall backup + backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); + // set overall backup status: complete + backupInfo.setState(BackupState.COMPLETE); + backupInfo.setProgress(100); + // add and store the manifest for the backup + addManifest(backupInfo, backupManager, type, conf); + + // compose the backup complete data + String backupCompleteData = + obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs() + ",completets=" + + backupInfo.getCompleteTs() + ",bytescopied=" + backupInfo.getTotalBytesCopied(); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData); + } + + // when full backup is done: + // - delete HBase snapshot + // - clean up directories with prefix "exportSnapshot-", which are generated when exporting + // snapshots + if (type == BackupType.FULL) { + deleteSnapshots(conn, backupInfo, conf); + cleanupExportSnapshotLog(conf); + } else if (type == BackupType.INCREMENTAL) { + cleanupDistCpLog(backupInfo, conf); + } + BackupSystemTable.deleteSnapshot(conn); + backupManager.updateBackupInfo(backupInfo); + + // Finish active session + backupManager.finishBackupSession(); + + LOG.info("Backup " + backupInfo.getBackupId() + " completed."); + } + + /** + * Backup request execution. + * @throws IOException if the execution of the backup fails + */ + public abstract void execute() throws IOException; + + protected Stage getTestStage() { + return Stage.valueOf("stage_" + conf.getInt(BACKUP_TEST_MODE_STAGE, 0)); + } + + protected void failStageIf(Stage stage) throws IOException { + Stage current = getTestStage(); + if (current == stage) { + throw new IOException("Failed stage " + stage + " in testing"); + } + } + + public enum Stage { + stage_0, + stage_1, + stage_2, + stage_3, + stage_4 + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java new file mode 100644 index 000000000000..51a276df4c5a --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java @@ -0,0 +1,428 @@ +/* + * 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.mapreduce; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.snapshot.ExportSnapshot; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Cluster; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.tools.CopyListingFileStatus; +import org.apache.hadoop.tools.DistCp; +import org.apache.hadoop.tools.DistCpConstants; +import org.apache.hadoop.tools.DistCpOptions; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Map-Reduce implementation of {@link BackupCopyJob}. Basically, there are 2 types of copy + * operation: one is copying from snapshot, which bases on extending ExportSnapshot's function, the + * other is copying for incremental log files, which bases on extending DistCp's function. + */ +@InterfaceAudience.Private +public class MapReduceBackupCopyJob implements BackupCopyJob { + public static final String NUMBER_OF_LEVELS_TO_PRESERVE_KEY = "num.levels.preserve"; + private static final Logger LOG = LoggerFactory.getLogger(MapReduceBackupCopyJob.class); + + private Configuration conf; + + // Accumulated progress within the whole backup process for the copy operation + private float progressDone = 0.1f; + private long bytesCopied = 0; + private static float INIT_PROGRESS = 0.1f; + + // The percentage of the current copy task within the whole task if multiple time copies are + // needed. The default value is 100%, which means only 1 copy task for the whole. + private float subTaskPercntgInWholeTask = 1f; + + public MapReduceBackupCopyJob() { + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + /** + * Get the current copy task percentage within the whole task if multiple copies are needed. + * @return the current copy task percentage + */ + public float getSubTaskPercntgInWholeTask() { + return subTaskPercntgInWholeTask; + } + + /** + * Set the current copy task percentage within the whole task if multiple copies are needed. Must + * be called before calling + * {@link #copy(BackupInfo, BackupManager, Configuration, BackupType, String[])} + * @param subTaskPercntgInWholeTask The percentage of the copy subtask + */ + public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) { + this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask; + } + + static class SnapshotCopy extends ExportSnapshot { + private BackupInfo backupInfo; + private TableName table; + + public SnapshotCopy(BackupInfo backupInfo, TableName table) { + super(); + this.backupInfo = backupInfo; + this.table = table; + } + + public TableName getTable() { + return this.table; + } + + public BackupInfo getBackupInfo() { + return this.backupInfo; + } + } + + /** + * Update the ongoing backup with new progress. + * @param backupInfo backup info + * @param newProgress progress + * @param bytesCopied bytes copied + * @throws IOException exception + */ + static void updateProgress(BackupInfo backupInfo, BackupManager backupManager, int newProgress, + long bytesCopied) throws IOException { + // compose the new backup progress data, using fake number for now + String backupProgressData = newProgress + "%"; + + backupInfo.setProgress(newProgress); + backupManager.updateBackupInfo(backupInfo); + LOG.debug("Backup progress data \"" + backupProgressData + + "\" has been updated to backup system table for " + backupInfo.getBackupId()); + } + + /** + * Extends DistCp for progress updating to backup system table during backup. Using DistCpV2 + * (MAPREDUCE-2765). Simply extend it and override execute() method to get the Job reference for + * progress updating. Only the argument "src1, [src2, [...]] dst" is supported, no more DistCp + * options. + */ + + class BackupDistCp extends DistCp { + + private BackupInfo backupInfo; + private BackupManager backupManager; + + public BackupDistCp(Configuration conf, DistCpOptions options, BackupInfo backupInfo, + BackupManager backupManager) throws Exception { + super(conf, options); + this.backupInfo = backupInfo; + this.backupManager = backupManager; + } + + @Override + public Job execute() throws Exception { + + // reflection preparation for private methods and fields + Class classDistCp = org.apache.hadoop.tools.DistCp.class; + Method methodCleanup = classDistCp.getDeclaredMethod("cleanup"); + + Field fieldInputOptions = getInputOptionsField(classDistCp); + Field fieldSubmitted = classDistCp.getDeclaredField("submitted"); + + methodCleanup.setAccessible(true); + fieldInputOptions.setAccessible(true); + fieldSubmitted.setAccessible(true); + + // execute() logic starts here + assert fieldInputOptions.get(this) != null; + + Job job = null; + try { + + List srcs = getSourcePaths(fieldInputOptions); + + long totalSrcLgth = 0; + for (Path aSrc : srcs) { + totalSrcLgth += BackupUtils.getFilesLength(aSrc.getFileSystem(super.getConf()), aSrc); + } + + // Async call + job = super.execute(); + // Update the copy progress to system table every 0.5s if progress value changed + int progressReportFreq = MapReduceBackupCopyJob.this.getConf() + .getInt("hbase.backup.progressreport.frequency", 500); + float lastProgress = progressDone; + while (!job.isComplete()) { + float newProgress = + progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS); + + if (newProgress > lastProgress) { + + BigDecimal progressData = + new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP); + String newProgressStr = progressData + "%"; + LOG.info("Progress: " + newProgressStr); + updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied); + LOG.debug("Backup progress data updated to backup system table: \"Progress: " + + newProgressStr + ".\""); + lastProgress = newProgress; + } + Thread.sleep(progressReportFreq); + } + // update the progress data after copy job complete + float newProgress = + progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS); + BigDecimal progressData = + new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP); + + String newProgressStr = progressData + "%"; + LOG.info("Progress: " + newProgressStr + " subTask: " + subTaskPercntgInWholeTask + + " mapProgress: " + job.mapProgress()); + + // accumulate the overall backup progress + progressDone = newProgress; + bytesCopied += totalSrcLgth; + + updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied); + LOG.debug("Backup progress data updated to backup system table: \"Progress: " + + newProgressStr + " - " + bytesCopied + " bytes copied.\""); + } catch (Throwable t) { + LOG.error(t.toString(), t); + throw t; + } + + String jobID = job.getJobID().toString(); + job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID); + + LOG.debug( + "DistCp job-id: " + jobID + " completed: " + job.isComplete() + " " + job.isSuccessful()); + Counters ctrs = job.getCounters(); + LOG.debug(Objects.toString(ctrs)); + if (job.isComplete() && !job.isSuccessful()) { + throw new Exception("DistCp job-id: " + jobID + " failed"); + } + + return job; + } + + private Field getInputOptionsField(Class classDistCp) throws IOException { + Field f = null; + try { + f = classDistCp.getDeclaredField("inputOptions"); + } catch (Exception e) { + // Haddop 3 + try { + f = classDistCp.getDeclaredField("context"); + } catch (NoSuchFieldException | SecurityException e1) { + throw new IOException(e1); + } + } + return f; + } + + @SuppressWarnings("unchecked") + private List getSourcePaths(Field fieldInputOptions) throws IOException { + Object options; + try { + options = fieldInputOptions.get(this); + if (options instanceof DistCpOptions) { + return ((DistCpOptions) options).getSourcePaths(); + } else { + // Hadoop 3 + Class classContext = Class.forName("org.apache.hadoop.tools.DistCpContext"); + Method methodGetSourcePaths = classContext.getDeclaredMethod("getSourcePaths"); + methodGetSourcePaths.setAccessible(true); + + return (List) methodGetSourcePaths.invoke(options); + } + } catch (IllegalArgumentException | IllegalAccessException | ClassNotFoundException + | NoSuchMethodException | SecurityException | InvocationTargetException e) { + throw new IOException(e); + } + + } + + @Override + protected Path createInputFileListing(Job job) throws IOException { + + if (conf.get(NUMBER_OF_LEVELS_TO_PRESERVE_KEY) == null) { + return super.createInputFileListing(job); + } + long totalBytesExpected = 0; + int totalRecords = 0; + Path fileListingPath = getFileListingPath(); + try (SequenceFile.Writer writer = getWriter(fileListingPath)) { + List srcFiles = getSourceFiles(); + if (srcFiles.size() == 0) { + return fileListingPath; + } + totalRecords = srcFiles.size(); + FileSystem fs = srcFiles.get(0).getFileSystem(conf); + for (Path path : srcFiles) { + FileStatus fst = fs.getFileStatus(path); + totalBytesExpected += fst.getLen(); + Text key = getKey(path); + writer.append(key, new CopyListingFileStatus(fst)); + } + writer.close(); + + // update jobs configuration + + Configuration cfg = job.getConfiguration(); + cfg.setLong(DistCpConstants.CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED, totalBytesExpected); + cfg.set(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, fileListingPath.toString()); + cfg.setLong(DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS, totalRecords); + } catch (NoSuchFieldException | SecurityException | IllegalArgumentException + | IllegalAccessException | NoSuchMethodException | ClassNotFoundException + | InvocationTargetException e) { + throw new IOException(e); + } + return fileListingPath; + } + + private Text getKey(Path path) { + int level = conf.getInt(NUMBER_OF_LEVELS_TO_PRESERVE_KEY, 1); + int count = 0; + String relPath = ""; + while (count++ < level) { + relPath = Path.SEPARATOR + path.getName() + relPath; + path = path.getParent(); + } + return new Text(relPath); + } + + private List getSourceFiles() throws NoSuchFieldException, SecurityException, + IllegalArgumentException, IllegalAccessException, NoSuchMethodException, + ClassNotFoundException, InvocationTargetException, IOException { + Field options = null; + try { + options = DistCp.class.getDeclaredField("inputOptions"); + } catch (NoSuchFieldException | SecurityException e) { + options = DistCp.class.getDeclaredField("context"); + } + options.setAccessible(true); + return getSourcePaths(options); + } + + private SequenceFile.Writer getWriter(Path pathToListFile) throws IOException { + FileSystem fs = pathToListFile.getFileSystem(conf); + fs.delete(pathToListFile, false); + return SequenceFile.createWriter(conf, SequenceFile.Writer.file(pathToListFile), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(CopyListingFileStatus.class), + SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE)); + } + + } + + /** + * Do backup copy based on different types. + * @param context The backup info + * @param conf The hadoop configuration + * @param copyType The backup copy type + * @param options Options for customized ExportSnapshot or DistCp + * @throws IOException exception + */ + @Override + public int copy(BackupInfo context, BackupManager backupManager, Configuration conf, + BackupType copyType, String[] options) throws IOException { + int res = 0; + + try { + if (copyType == BackupType.FULL) { + SnapshotCopy snapshotCp = new SnapshotCopy(context, context.getTableBySnapshot(options[1])); + LOG.debug("Doing SNAPSHOT_COPY"); + // Make a new instance of conf to be used by the snapshot copy class. + snapshotCp.setConf(new Configuration(conf)); + res = snapshotCp.run(options); + + } else if (copyType == BackupType.INCREMENTAL) { + LOG.debug("Doing COPY_TYPE_DISTCP"); + setSubTaskPercntgInWholeTask(1f); + + BackupDistCp distcp = + new BackupDistCp(new Configuration(conf), null, context, backupManager); + // Handle a special case where the source file is a single file. + // In this case, distcp will not create the target dir. It just take the + // target as a file name and copy source file to the target (as a file name). + // We need to create the target dir before run distcp. + LOG.debug("DistCp options: " + Arrays.toString(options)); + Path dest = new Path(options[options.length - 1]); + String[] newOptions = new String[options.length + 1]; + System.arraycopy(options, 0, newOptions, 1, options.length); + newOptions[0] = "-async"; // run DisCp in async mode + FileSystem destfs = dest.getFileSystem(conf); + if (!destfs.exists(dest)) { + destfs.mkdirs(dest); + } + res = distcp.run(newOptions); + } + return res; + + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public void cancel(String jobId) throws IOException { + JobID id = JobID.forName(jobId); + Cluster cluster = new Cluster(this.getConf()); + try { + Job job = cluster.getJob(id); + if (job == null) { + LOG.error("No job found for " + id); + // should we throw exception + return; + } + if (job.isComplete() || job.isRetired()) { + return; + } + + job.killJob(); + LOG.debug("Killed copy job " + id); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java new file mode 100644 index 000000000000..3b4cf0246d73 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java @@ -0,0 +1,399 @@ +/* + * 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.mapreduce; + +import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupMergeJob; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.util.Tool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * MapReduce implementation of {@link BackupMergeJob} Must be initialized with configuration of a + * backup destination cluster + */ +@InterfaceAudience.Private +public class MapReduceBackupMergeJob implements BackupMergeJob { + public static final Logger LOG = LoggerFactory.getLogger(MapReduceBackupMergeJob.class); + + protected Tool player; + protected Configuration conf; + + public MapReduceBackupMergeJob() { + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public void run(String[] backupIds) throws IOException { + String bulkOutputConfKey; + + // TODO : run player on remote cluster + player = new MapReduceHFileSplitterJob(); + bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY; + // Player reads all files in arbitrary directory structure and creates + // a Map task for each file + String bids = StringUtils.join(backupIds, ","); + + if (LOG.isDebugEnabled()) { + LOG.debug("Merge backup images " + bids); + } + + List> processedTableList = new ArrayList<>(); + boolean finishedTables = false; + Connection conn = ConnectionFactory.createConnection(getConf()); + BackupSystemTable table = new BackupSystemTable(conn); + FileSystem fs = FileSystem.get(getConf()); + + try { + + // Get exclusive lock on backup system + table.startBackupExclusiveOperation(); + // Start merge operation + table.startMergeOperation(backupIds); + + // Select most recent backup id + String mergedBackupId = BackupUtils.findMostRecentBackupId(backupIds); + + TableName[] tableNames = getTableNamesInBackupImages(backupIds); + + BackupInfo bInfo = table.readBackupInfo(backupIds[0]); + String backupRoot = bInfo.getBackupRootDir(); + + for (int i = 0; i < tableNames.length; i++) { + LOG.info("Merge backup images for " + tableNames[i]); + + // Find input directories for table + Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds); + String dirs = StringUtils.join(dirPaths, ","); + + Path bulkOutputPath = BackupUtils.getBulkOutputDir( + BackupUtils.getFileNameCompatibleString(tableNames[i]), getConf(), false); + // Delete content if exists + if (fs.exists(bulkOutputPath)) { + if (!fs.delete(bulkOutputPath, true)) { + LOG.warn("Can not delete: " + bulkOutputPath); + } + } + Configuration conf = getConf(); + conf.set(bulkOutputConfKey, bulkOutputPath.toString()); + String[] playerArgs = { dirs, tableNames[i].getNameAsString() }; + + player.setConf(getConf()); + int result = player.run(playerArgs); + if (!succeeded(result)) { + throw new IOException("Can not merge backup images for " + dirs + + " (check Hadoop/MR and HBase logs). Player return code =" + result); + } + // Add to processed table list + processedTableList.add(new Pair<>(tableNames[i], bulkOutputPath)); + LOG.debug("Merge Job finished:" + result); + } + List tableList = toTableNameList(processedTableList); + table.updateProcessedTablesForMerge(tableList); + finishedTables = true; + + // PHASE 2 (modification of a backup file system) + // Move existing mergedBackupId data into tmp directory + // we will need it later in case of a failure + Path tmpBackupDir = + HBackupFileSystem.getBackupTmpDirPathForBackupId(backupRoot, mergedBackupId); + Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, mergedBackupId); + + if (!fs.rename(backupDirPath, tmpBackupDir)) { + throw new IOException("Failed to rename " + backupDirPath + " to " + tmpBackupDir); + } else { + LOG.debug("Renamed " + backupDirPath + " to " + tmpBackupDir); + } + // Move new data into backup dest + for (Pair tn : processedTableList) { + moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId); + } + // Update backup manifest + List backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId); + updateBackupManifest(tmpBackupDir.getParent().toString(), mergedBackupId, backupsToDelete); + // Copy meta files back from tmp to backup dir + copyMetaData(fs, tmpBackupDir, backupDirPath); + // Delete tmp dir (Rename back during repair) + if (!fs.delete(tmpBackupDir, true)) { + // WARN and ignore + LOG.warn("Could not delete tmp dir: " + tmpBackupDir); + } + // Delete old data + deleteBackupImages(backupsToDelete, conn, fs, backupRoot); + // Finish merge session + table.finishMergeOperation(); + // Release lock + table.finishBackupExclusiveOperation(); + } catch (RuntimeException e) { + + throw e; + } catch (Exception e) { + LOG.error(e.toString(), e); + if (!finishedTables) { + // cleanup bulk directories and finish merge + // merge MUST be repeated (no need for repair) + cleanupBulkLoadDirs(fs, toPathList(processedTableList)); + table.finishMergeOperation(); + table.finishBackupExclusiveOperation(); + throw new IOException("Backup merge operation failed, you should try it again", e); + } else { + // backup repair must be run + throw new IOException( + "Backup merge operation failed, run backup repair tool to restore system's integrity", e); + } + } finally { + table.close(); + conn.close(); + } + } + + /** + * Copy meta data to of a backup session + * @param fs file system + * @param tmpBackupDir temp backup directory, where meta is locaed + * @param backupDirPath new path for backup + * @throws IOException exception + */ + protected void copyMetaData(FileSystem fs, Path tmpBackupDir, Path backupDirPath) + throws IOException { + RemoteIterator it = fs.listFiles(tmpBackupDir, true); + List toKeep = new ArrayList(); + while (it.hasNext()) { + Path p = it.next().getPath(); + if (fs.isDirectory(p)) { + continue; + } + // Keep meta + String fileName = p.toString(); + if ( + fileName.indexOf(FSTableDescriptors.TABLEINFO_DIR) > 0 + || fileName.indexOf(HRegionFileSystem.REGION_INFO_FILE) > 0 + ) { + toKeep.add(p); + } + } + // Copy meta to destination + for (Path p : toKeep) { + Path newPath = convertToDest(p, backupDirPath); + copyFile(fs, p, newPath); + } + } + + /** + * Copy file in DFS from p to newPath + * @param fs file system + * @param p old path + * @param newPath new path + * @throws IOException exception + */ + protected void copyFile(FileSystem fs, Path p, Path newPath) throws IOException { + try (InputStream in = fs.open(p); OutputStream out = fs.create(newPath, true)) { + IOUtils.copy(in, out); + } + boolean exists = fs.exists(newPath); + if (!exists) { + throw new IOException("Failed to copy meta file to: " + newPath); + } + } + + /** + * Converts path before copying + * @param p path + * @param backupDirPath backup root + * @return converted path + */ + protected Path convertToDest(Path p, Path backupDirPath) { + String backupId = backupDirPath.getName(); + Deque stack = new ArrayDeque(); + String name = null; + while (true) { + name = p.getName(); + if (!name.equals(backupId)) { + stack.push(name); + p = p.getParent(); + } else { + break; + } + } + Path newPath = new Path(backupDirPath.toString()); + while (!stack.isEmpty()) { + newPath = new Path(newPath, stack.pop()); + } + return newPath; + } + + protected List toPathList(List> processedTableList) { + ArrayList list = new ArrayList<>(); + for (Pair p : processedTableList) { + list.add(p.getSecond()); + } + return list; + } + + protected List toTableNameList(List> processedTableList) { + ArrayList list = new ArrayList<>(); + for (Pair p : processedTableList) { + list.add(p.getFirst()); + } + return list; + } + + protected void cleanupBulkLoadDirs(FileSystem fs, List pathList) throws IOException { + for (Path path : pathList) { + if (!fs.delete(path, true)) { + LOG.warn("Can't delete " + path); + } + } + } + + protected void updateBackupManifest(String backupRoot, String mergedBackupId, + List backupsToDelete) throws IllegalArgumentException, IOException { + BackupManifest manifest = + HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId); + manifest.getBackupImage().removeAncestors(backupsToDelete); + // save back + manifest.store(conf); + } + + protected void deleteBackupImages(List backupIds, Connection conn, FileSystem fs, + String backupRoot) throws IOException { + // Delete from backup system table + try (BackupSystemTable table = new BackupSystemTable(conn)) { + for (String backupId : backupIds) { + table.deleteBackupInfo(backupId); + } + } + + // Delete from file system + for (String backupId : backupIds) { + Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId); + + if (!fs.delete(backupDirPath, true)) { + LOG.warn("Could not delete " + backupDirPath); + } + } + } + + protected List getBackupIdsToDelete(String[] backupIds, String mergedBackupId) { + List list = new ArrayList<>(); + for (String id : backupIds) { + if (id.equals(mergedBackupId)) { + continue; + } + list.add(id); + } + return list; + } + + protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, + TableName tableName, String mergedBackupId) throws IllegalArgumentException, IOException { + Path dest = + new Path(HBackupFileSystem.getTableBackupDir(backupRoot, mergedBackupId, tableName)); + + FileStatus[] fsts = fs.listStatus(bulkOutputPath); + for (FileStatus fst : fsts) { + if (fst.isDirectory()) { + String family = fst.getPath().getName(); + Path newDst = new Path(dest, family); + if (fs.exists(newDst)) { + if (!fs.delete(newDst, true)) { + throw new IOException("failed to delete :" + newDst); + } + } else { + fs.mkdirs(dest); + } + boolean result = fs.rename(fst.getPath(), dest); + LOG.debug("MoveData from " + fst.getPath() + " to " + dest + " result=" + result); + } + } + } + + protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException { + Set allSet = new HashSet<>(); + + try (Connection conn = ConnectionFactory.createConnection(conf); + BackupSystemTable table = new BackupSystemTable(conn)) { + for (String backupId : backupIds) { + BackupInfo bInfo = table.readBackupInfo(backupId); + + allSet.addAll(bInfo.getTableNames()); + } + } + + TableName[] ret = new TableName[allSet.size()]; + return allSet.toArray(ret); + } + + protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName, + String[] backupIds) throws IOException { + List dirs = new ArrayList<>(); + + for (String backupId : backupIds) { + Path fileBackupDirPath = + new Path(HBackupFileSystem.getTableBackupDir(backupRoot, backupId, tableName)); + if (fs.exists(fileBackupDirPath)) { + dirs.add(fileBackupDirPath); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("File: " + fileBackupDirPath + " does not exist."); + } + } + } + Path[] ret = new Path[dirs.size()]; + return dirs.toArray(ret); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java new file mode 100644 index 000000000000..766a99d778b8 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java @@ -0,0 +1,169 @@ +/* + * 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.mapreduce; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.CellSortReducer; +import org.apache.hadoop.hbase.mapreduce.HFileInputFormat; +import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.MapReduceExtendedCell; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles + * for later bulk importing. + */ +@InterfaceAudience.Private +public class MapReduceHFileSplitterJob extends Configured implements Tool { + private static final Logger LOG = LoggerFactory.getLogger(MapReduceHFileSplitterJob.class); + final static String NAME = "HFileSplitterJob"; + public final static String BULK_OUTPUT_CONF_KEY = "hfile.bulk.output"; + public final static String TABLES_KEY = "hfile.input.tables"; + public final static String TABLE_MAP_KEY = "hfile.input.tablesmap"; + private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; + + public MapReduceHFileSplitterJob() { + } + + protected MapReduceHFileSplitterJob(final Configuration c) { + super(c); + } + + /** + * A mapper that just writes out cells. This one can be used together with {@link CellSortReducer} + */ + static class HFileCellMapper extends Mapper { + + @Override + public void map(NullWritable key, Cell value, Context context) + throws IOException, InterruptedException { + context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), + new MapReduceExtendedCell(value)); + } + + @Override + public void setup(Context context) throws IOException { + // do nothing + } + } + + /** + * Sets up the actual job. + * @param args The command line parameters. + * @return The newly created job. + * @throws IOException When setting up the job fails. + */ + public Job createSubmittableJob(String[] args) throws IOException { + Configuration conf = getConf(); + String inputDirs = args[0]; + String tabName = args[1]; + conf.setStrings(TABLES_KEY, tabName); + conf.set(FileInputFormat.INPUT_DIR, inputDirs); + Job job = Job.getInstance(conf, + conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime())); + job.setJarByClass(MapReduceHFileSplitterJob.class); + job.setInputFormatClass(HFileInputFormat.class); + job.setMapOutputKeyClass(ImmutableBytesWritable.class); + String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); + if (hfileOutPath != null) { + LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs); + TableName tableName = TableName.valueOf(tabName); + job.setMapperClass(HFileCellMapper.class); + job.setReducerClass(CellSortReducer.class); + Path outputDir = new Path(hfileOutPath); + FileOutputFormat.setOutputPath(job, outputDir); + job.setMapOutputValueClass(MapReduceExtendedCell.class); + try (Connection conn = ConnectionFactory.createConnection(conf); + Table table = conn.getTable(tableName); + RegionLocator regionLocator = conn.getRegionLocator(tableName)) { + HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); + } + LOG.debug("success configuring load incremental job"); + + TableMapReduceUtil.addDependencyJars(job.getConfiguration(), + org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class); + } else { + throw new IOException("No bulk output directory specified"); + } + return job; + } + + /** + * Print usage + * @param errorMsg Error message. Can be null. + */ + private void usage(final String errorMsg) { + if (errorMsg != null && errorMsg.length() > 0) { + System.err.println("ERROR: " + errorMsg); + } + System.err.println("Usage: " + NAME + " [options] "); + System.err.println("Read all HFile's for
and split them to
region boundaries."); + System.err.println("
table to load.\n"); + System.err.println("To generate HFiles for a bulk data load, pass the option:"); + System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output"); + System.err.println("Other options:"); + System.err.println(" -D " + JOB_NAME_CONF_KEY + + "=jobName - use the specified mapreduce job name for the HFile splitter"); + System.err.println("For performance also consider the following options:\n" + + " -Dmapreduce.map.speculative=false\n" + " -Dmapreduce.reduce.speculative=false"); + } + + /** + * Main entry point. + * @param args The command line parameters. + * @throws Exception When running the job fails. + */ + public static void main(String[] args) throws Exception { + int ret = ToolRunner.run(new MapReduceHFileSplitterJob(HBaseConfiguration.create()), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length < 2) { + usage("Wrong number of arguments: " + args.length); + return -1; + } + Job job = createSubmittableJob(args); + int result = job.waitForCompletion(true) ? 0 : 1; + return result; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java new file mode 100644 index 000000000000..e6046bf5fb92 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java @@ -0,0 +1,119 @@ +/* + * 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.mapreduce; + +import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded; + +import java.io.IOException; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.RestoreJob; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.util.Tool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * MapReduce implementation of {@link RestoreJob} For backup restore, it runs + * {@link MapReduceHFileSplitterJob} job and creates HFiles which are aligned with a region + * boundaries of a table being restored. The resulting HFiles then are loaded using HBase bulk load + * tool {@link BulkLoadHFiles}. + */ +@InterfaceAudience.Private +public class MapReduceRestoreJob implements RestoreJob { + public static final Logger LOG = LoggerFactory.getLogger(MapReduceRestoreJob.class); + + private Tool player; + private Configuration conf; + + public MapReduceRestoreJob() { + } + + @Override + public void run(Path[] dirPaths, TableName[] tableNames, TableName[] newTableNames, + boolean fullBackupRestore) throws IOException { + String bulkOutputConfKey; + + player = new MapReduceHFileSplitterJob(); + bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY; + // Player reads all files in arbitrary directory structure and creates + // a Map task for each file + String dirs = StringUtils.join(dirPaths, ","); + + if (LOG.isDebugEnabled()) { + LOG.debug("Restore " + (fullBackupRestore ? "full" : "incremental") + + " backup from directory " + dirs + " from hbase tables " + + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND) + + " to tables " + + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)); + } + + for (int i = 0; i < tableNames.length; i++) { + LOG.info("Restore " + tableNames[i] + " into " + newTableNames[i]); + + Path bulkOutputPath = BackupUtils + .getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]), getConf()); + Configuration conf = getConf(); + conf.set(bulkOutputConfKey, bulkOutputPath.toString()); + String[] playerArgs = { dirs, + fullBackupRestore ? newTableNames[i].getNameAsString() : tableNames[i].getNameAsString() }; + + int result; + try { + + player.setConf(getConf()); + result = player.run(playerArgs); + if (succeeded(result)) { + // do bulk load + BulkLoadHFiles loader = BackupUtils.createLoader(getConf()); + if (LOG.isDebugEnabled()) { + LOG.debug("Restoring HFiles from directory " + bulkOutputPath); + } + + if (loader.bulkLoad(newTableNames[i], bulkOutputPath).isEmpty()) { + throw new IOException("Can not restore from backup directory " + dirs + + " (check Hadoop and HBase logs). Bulk loader returns null"); + } + } else { + throw new IOException("Can not restore from backup directory " + dirs + + " (check Hadoop/MR and HBase logs). Player return code =" + result); + } + LOG.debug("Restore Job finished:" + result); + } catch (Exception e) { + LOG.error(e.toString(), e); + throw new IOException( + "Can not restore from backup directory " + dirs + " (check Hadoop and HBase logs) ", e); + } + } + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java new file mode 100644 index 000000000000..391c7d9b1914 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java @@ -0,0 +1,179 @@ +/* + * 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.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate; +import org.apache.hadoop.hbase.net.Address; +import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils; +import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils; + +/** + * Implementation of a log cleaner that checks if a log is still scheduled for incremental backup + * before deleting it when its TTL is over. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class BackupLogCleaner extends BaseLogCleanerDelegate { + private static final Logger LOG = LoggerFactory.getLogger(BackupLogCleaner.class); + + private boolean stopped = false; + private Connection conn; + + public BackupLogCleaner() { + } + + @Override + public void init(Map params) { + MasterServices master = (MasterServices) MapUtils.getObject(params, HMaster.MASTER); + if (master != null) { + conn = master.getConnection(); + if (getConf() == null) { + super.setConf(conn.getConfiguration()); + } + } + if (conn == null) { + try { + conn = ConnectionFactory.createConnection(getConf()); + } catch (IOException ioe) { + throw new RuntimeException("Failed to create connection", ioe); + } + } + } + + private Map getServersToOldestBackupMapping(List backups) + throws IOException { + Map serverAddressToLastBackupMap = new HashMap<>(); + + Map tableNameBackupInfoMap = new HashMap<>(); + for (BackupInfo backupInfo : backups) { + for (TableName table : backupInfo.getTables()) { + tableNameBackupInfoMap.putIfAbsent(table, backupInfo.getStartTs()); + if (tableNameBackupInfoMap.get(table) <= backupInfo.getStartTs()) { + tableNameBackupInfoMap.put(table, backupInfo.getStartTs()); + for (Map.Entry entry : backupInfo.getTableSetTimestampMap().get(table) + .entrySet()) { + serverAddressToLastBackupMap.put(Address.fromString(entry.getKey()), entry.getValue()); + } + } + } + } + + return serverAddressToLastBackupMap; + } + + @Override + public Iterable getDeletableFiles(Iterable files) { + List filteredFiles = new ArrayList<>(); + + // all members of this class are null if backup is disabled, + // so we cannot filter the files + if (this.getConf() == null || !BackupManager.isBackupEnabled(getConf())) { + LOG.debug("Backup is not enabled. Check your {} setting", + BackupRestoreConstants.BACKUP_ENABLE_KEY); + return files; + } + + Map addressToLastBackupMap; + try { + try (BackupManager backupManager = new BackupManager(conn, getConf())) { + addressToLastBackupMap = + getServersToOldestBackupMapping(backupManager.getBackupHistory(true)); + } + } catch (IOException ex) { + LOG.error("Failed to analyse backup history with exception: {}. Retaining all logs", + ex.getMessage(), ex); + return Collections.emptyList(); + } + for (FileStatus file : files) { + String fn = file.getPath().getName(); + if (fn.startsWith(WALProcedureStore.LOG_PREFIX)) { + filteredFiles.add(file); + continue; + } + + try { + Address walServerAddress = + Address.fromString(BackupUtils.parseHostNameFromLogFile(file.getPath())); + long walTimestamp = WAL.getTimestamp(file.getPath().getName()); + + if ( + !addressToLastBackupMap.containsKey(walServerAddress) + || addressToLastBackupMap.get(walServerAddress) >= walTimestamp + ) { + filteredFiles.add(file); + } + } catch (Exception ex) { + LOG.warn( + "Error occurred while filtering file: {} with error: {}. Ignoring cleanup of this log", + file.getPath(), ex.getMessage()); + } + } + + LOG.info("Total files: {}, Filtered Files: {}", IterableUtils.size(files), + filteredFiles.size()); + return filteredFiles; + } + + @Override + public void setConf(Configuration config) { + // If backup is disabled, keep all members null + super.setConf(config); + if ( + !config.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.BACKUP_ENABLE_DEFAULT) + ) { + LOG.warn("Backup is disabled - allowing all wals to be deleted"); + } + } + + @Override + public void stop(String why) { + if (!this.stopped) { + this.stopped = true; + LOG.info("Stopping BackupLogCleaner"); + } + } + + @Override + public boolean isStopped() { + return this.stopped; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java new file mode 100644 index 000000000000..12b0bda56152 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java @@ -0,0 +1,173 @@ +/* + * 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.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadPoolExecutor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.errorhandling.ForeignException; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.MetricsMaster; +import org.apache.hadoop.hbase.procedure.MasterProcedureManager; +import org.apache.hadoop.hbase.procedure.Procedure; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinationManager; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; +import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinationManager; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.access.AccessChecker; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; + +/** + * Master procedure manager for coordinated cluster-wide WAL roll operation, which is run during + * backup operation, see {@link MasterProcedureManager} and {@link RegionServerProcedureManager} + */ +@InterfaceAudience.Private +public class LogRollMasterProcedureManager extends MasterProcedureManager { + private static final Logger LOG = LoggerFactory.getLogger(LogRollMasterProcedureManager.class); + + public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc"; + public static final String ROLLLOG_PROCEDURE_NAME = "rolllog"; + public static final String BACKUP_WAKE_MILLIS_KEY = "hbase.backup.logroll.wake.millis"; + public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.logroll.timeout.millis"; + public static final String BACKUP_POOL_THREAD_NUMBER_KEY = + "hbase.backup.logroll.pool.thread.number"; + + public static final int BACKUP_WAKE_MILLIS_DEFAULT = 500; + public static final int BACKUP_TIMEOUT_MILLIS_DEFAULT = 180000; + public static final int BACKUP_POOL_THREAD_NUMBER_DEFAULT = 8; + private MasterServices master; + private ProcedureCoordinator coordinator; + private boolean done; + + @Override + public void stop(String why) { + LOG.info("stop: " + why); + } + + @Override + public boolean isStopped() { + return false; + } + + @Override + public void initialize(MasterServices master, MetricsMaster metricsMaster) + throws IOException, UnsupportedOperationException { + this.master = master; + this.done = false; + + // setup the default procedure coordinator + String name = master.getServerName().toString(); + + // get the configuration for the coordinator + Configuration conf = master.getConfiguration(); + long wakeFrequency = conf.getInt(BACKUP_WAKE_MILLIS_KEY, BACKUP_WAKE_MILLIS_DEFAULT); + long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT); + int opThreads = conf.getInt(BACKUP_POOL_THREAD_NUMBER_KEY, BACKUP_POOL_THREAD_NUMBER_DEFAULT); + + // setup the default procedure coordinator + ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, opThreads); + ProcedureCoordinationManager coordManager = new ZKProcedureCoordinationManager(master); + ProcedureCoordinatorRpcs comms = + coordManager.getProcedureCoordinatorRpcs(getProcedureSignature(), name); + this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); + + } + + @Override + public String getProcedureSignature() { + return ROLLLOG_PROCEDURE_SIGNATURE; + } + + @Override + public void execProcedure(ProcedureDescription desc) throws IOException { + if (!isBackupEnabled()) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + this.done = false; + // start the process on the RS + ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance()); + List serverNames = master.getServerManager().getOnlineServersList(); + List servers = new ArrayList<>(); + for (ServerName sn : serverNames) { + servers.add(sn.toString()); + } + + List conf = desc.getConfigurationList(); + byte[] data = new byte[0]; + if (conf.size() > 0) { + // Get backup root path + data = Bytes.toBytes(conf.get(0).getValue()); + } + Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), data, servers); + if (proc == null) { + String msg = "Failed to submit distributed procedure for '" + desc.getInstance() + "'"; + LOG.error(msg); + throw new IOException(msg); + } + + try { + // wait for the procedure to complete. A timer thread is kicked off that should cancel this + // if it takes too long. + proc.waitForCompleted(); + LOG.info("Done waiting - exec procedure for " + desc.getInstance()); + LOG.info("Distributed roll log procedure is successful!"); + this.done = true; + } catch (InterruptedException e) { + ForeignException ee = + new ForeignException("Interrupted while waiting for roll log procdure to finish", e); + monitor.receive(ee); + Thread.currentThread().interrupt(); + } catch (ForeignException e) { + ForeignException ee = + new ForeignException("Exception while waiting for roll log procdure to finish", e); + monitor.receive(ee); + } + monitor.rethrowException(); + } + + @Override + public void checkPermissions(ProcedureDescription desc, AccessChecker accessChecker, User user) + throws IOException { + // TODO: what permissions checks are needed here? + } + + private boolean isBackupEnabled() { + return BackupManager.isBackupEnabled(master.getConfiguration()); + } + + @Override + public boolean isProcedureDone(ProcedureDescription desc) { + return done; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java new file mode 100644 index 000000000000..5b21feeba75f --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java @@ -0,0 +1,167 @@ +/* + * 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.regionserver; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.Callable; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.errorhandling.ForeignException; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.procedure.ProcedureMember; +import org.apache.hadoop.hbase.procedure.Subprocedure; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This backup sub-procedure implementation forces a WAL rolling on a RS. + */ +@InterfaceAudience.Private +public class LogRollBackupSubprocedure extends Subprocedure { + private static final Logger LOG = LoggerFactory.getLogger(LogRollBackupSubprocedure.class); + + private final RegionServerServices rss; + private final LogRollBackupSubprocedurePool taskManager; + private String backupRoot; + + public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member, + ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout, + LogRollBackupSubprocedurePool taskManager, byte[] data) { + super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener, + wakeFrequency, timeout); + LOG.info("Constructing a LogRollBackupSubprocedure."); + this.rss = rss; + this.taskManager = taskManager; + if (data != null) { + backupRoot = new String(data, StandardCharsets.UTF_8); + } + } + + /** + * Callable task. TODO. We don't need a thread pool to execute roll log. This can be simplified + * with no use of sub-procedure pool. + */ + class RSRollLogTask implements Callable { + RSRollLogTask() { + } + + @Override + public Void call() throws Exception { + if (LOG.isDebugEnabled()) { + LOG.debug("DRPC started: " + rss.getServerName()); + } + + AbstractFSWAL fsWAL = (AbstractFSWAL) rss.getWAL(null); + long filenum = fsWAL.getFilenum(); + List wals = rss.getWALs(); + long highest = -1; + for (WAL wal : wals) { + if (wal == null) { + continue; + } + + if (((AbstractFSWAL) wal).getFilenum() > highest) { + highest = ((AbstractFSWAL) wal).getFilenum(); + } + } + + LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum + + " highest: " + highest + " on " + rss.getServerName()); + ((HRegionServer) rss).getWalRoller().requestRollAll(); + long start = EnvironmentEdgeManager.currentTime(); + while (!((HRegionServer) rss).getWalRoller().walRollFinished()) { + Thread.sleep(20); + } + LOG.debug("log roll took " + (EnvironmentEdgeManager.currentTime() - start)); + LOG.info("After roll log in backup subprocedure, current log number: " + fsWAL.getFilenum() + + " on " + rss.getServerName()); + + Connection connection = rss.getConnection(); + try (final BackupSystemTable table = new BackupSystemTable(connection)) { + // sanity check, good for testing + HashMap serverTimestampMap = + table.readRegionServerLastLogRollResult(backupRoot); + String host = rss.getServerName().getHostname(); + int port = rss.getServerName().getPort(); + String server = host + ":" + port; + Long sts = serverTimestampMap.get(host); + if (sts != null && sts > highest) { + LOG + .warn("Won't update server's last roll log result: current=" + sts + " new=" + highest); + return null; + } + // write the log number to backup system table. + table.writeRegionServerLastLogRollResult(server, highest, backupRoot); + return null; + } catch (Exception e) { + LOG.error(e.toString(), e); + throw e; + } + } + } + + private void rolllog() throws ForeignException { + monitor.rethrowException(); + + taskManager.submitTask(new RSRollLogTask()); + monitor.rethrowException(); + + // wait for everything to complete. + taskManager.waitForOutstandingTasks(); + monitor.rethrowException(); + } + + @Override + public void acquireBarrier() { + // do nothing, executing in inside barrier step. + } + + /** + * do a log roll. + * @return some bytes + */ + @Override + public byte[] insideBarrier() throws ForeignException { + rolllog(); + return null; + } + + /** + * Cancel threads if they haven't finished. + */ + @Override + public void cleanup(Exception e) { + taskManager.abort("Aborting log roll subprocedure tasks for backup due to error", e); + } + + /** + * Hooray! + */ + public void releaseBarrier() { + // NO OP + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java new file mode 100644 index 000000000000..d6c39d7ed54f --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java @@ -0,0 +1,137 @@ +/* + * 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.regionserver; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.errorhandling.ForeignException; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * Handle running each of the individual tasks for completing a backup procedure on a region server. + */ +@InterfaceAudience.Private +public class LogRollBackupSubprocedurePool implements Closeable, Abortable { + private static final Logger LOG = LoggerFactory.getLogger(LogRollBackupSubprocedurePool.class); + + /** Maximum number of concurrent snapshot region tasks that can run concurrently */ + private static final String CONCURENT_BACKUP_TASKS_KEY = "hbase.backup.region.concurrentTasks"; + private static final int DEFAULT_CONCURRENT_BACKUP_TASKS = 3; + + private final ExecutorCompletionService taskPool; + private final ThreadPoolExecutor executor; + private volatile boolean aborted; + private final List> futures = new ArrayList<>(); + private final String name; + + public LogRollBackupSubprocedurePool(String name, Configuration conf) { + // configure the executor service + long keepAlive = conf.getLong(LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_KEY, + LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT); + int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS); + this.name = name; + executor = + new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").setDaemon(true) + .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); + taskPool = new ExecutorCompletionService<>(executor); + } + + /** + * Submit a task to the pool. + */ + public void submitTask(final Callable task) { + Future f = this.taskPool.submit(task); + futures.add(f); + } + + /** + * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)} + * @return true on success, false otherwise + * @throws ForeignException exception + */ + public boolean waitForOutstandingTasks() throws ForeignException { + LOG.debug("Waiting for backup procedure to finish."); + + try { + for (Future f : futures) { + f.get(); + } + return true; + } catch (InterruptedException e) { + if (aborted) { + throw new ForeignException("Interrupted and found to be aborted while waiting for tasks!", + e); + } + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + if (e.getCause() instanceof ForeignException) { + throw (ForeignException) e.getCause(); + } + throw new ForeignException(name, e.getCause()); + } finally { + // close off remaining tasks + for (Future f : futures) { + if (!f.isDone()) { + f.cancel(true); + } + } + } + return false; + } + + /** + * Attempt to cleanly shutdown any running tasks - allows currently running tasks to cleanly + * finish + */ + @Override + public void close() { + executor.shutdown(); + } + + @Override + public void abort(String why, Throwable e) { + if (this.aborted) { + return; + } + + this.aborted = true; + LOG.warn("Aborting because: " + why, e); + this.executor.shutdownNow(); + } + + @Override + public boolean isAborted() { + return this.aborted; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java new file mode 100644 index 000000000000..95b65f8e69cb --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java @@ -0,0 +1,175 @@ +/* + * 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.regionserver; + +import java.io.IOException; +import java.util.concurrent.ThreadPoolExecutor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinationManager; +import org.apache.hadoop.hbase.procedure.ProcedureMember; +import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs; +import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; +import org.apache.hadoop.hbase.procedure.Subprocedure; +import org.apache.hadoop.hbase.procedure.SubprocedureFactory; +import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinationManager; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This manager class handles the work dealing with distributed WAL roll request. + *

+ * This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is + * responsible by this region server. If any failures occur with the sub-procedure, the manager's + * procedure member notifies the procedure coordinator to abort all others. + *

+ * On startup, requires {@link #start()} to be called. + *

+ * On shutdown, requires org.apache.hadoop.hbase.procedure.ProcedureMember.close() to be called + */ +@InterfaceAudience.Private +public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager { + private static final Logger LOG = + LoggerFactory.getLogger(LogRollRegionServerProcedureManager.class); + + /** Conf key for number of request threads to start backup on region servers */ + public static final String BACKUP_REQUEST_THREADS_KEY = "hbase.backup.region.pool.threads"; + /** # of threads for backup work on the rs. */ + public static final int BACKUP_REQUEST_THREADS_DEFAULT = 10; + + public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.timeout"; + public static final long BACKUP_TIMEOUT_MILLIS_DEFAULT = 60000; + + /** Conf key for millis between checks to see if backup work completed or if there are errors */ + public static final String BACKUP_REQUEST_WAKE_MILLIS_KEY = "hbase.backup.region.wakefrequency"; + /** Default amount of time to check for errors while regions finish backup work */ + private static final long BACKUP_REQUEST_WAKE_MILLIS_DEFAULT = 500; + + private RegionServerServices rss; + private ProcedureMemberRpcs memberRpcs; + private ProcedureMember member; + private boolean started = false; + + /** + * Create a default backup procedure manager + */ + public LogRollRegionServerProcedureManager() { + } + + /** + * Start accepting backup procedure requests. + */ + @Override + public void start() { + if (!BackupManager.isBackupEnabled(rss.getConfiguration())) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + this.memberRpcs.start(rss.getServerName().toString(), member); + started = true; + LOG.info("Started region server backup manager."); + } + + /** + * Close this and all running backup procedure tasks + * @param force forcefully stop all running tasks + * @throws IOException exception + */ + @Override + public void stop(boolean force) throws IOException { + if (!started) { + return; + } + String mode = force ? "abruptly" : "gracefully"; + LOG.info("Stopping RegionServerBackupManager " + mode + "."); + + try { + this.member.close(); + } finally { + this.memberRpcs.close(); + } + } + + /** + * If in a running state, creates the specified subprocedure for handling a backup procedure. + * @return Subprocedure to submit to the ProcedureMember. + */ + public Subprocedure buildSubprocedure(byte[] data) { + // don't run a backup if the parent is stop(ping) + if (rss.isStopping() || rss.isStopped()) { + throw new IllegalStateException("Can't start backup procedure on RS: " + rss.getServerName() + + ", because stopping/stopped!"); + } + + LOG.info("Attempting to run a roll log procedure for backup."); + ForeignExceptionDispatcher errorDispatcher = new ForeignExceptionDispatcher(); + Configuration conf = rss.getConfiguration(); + long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT); + long wakeMillis = + conf.getLong(BACKUP_REQUEST_WAKE_MILLIS_KEY, BACKUP_REQUEST_WAKE_MILLIS_DEFAULT); + + LogRollBackupSubprocedurePool taskManager = + new LogRollBackupSubprocedurePool(rss.getServerName().toString(), conf); + return new LogRollBackupSubprocedure(rss, member, errorDispatcher, wakeMillis, timeoutMillis, + taskManager, data); + } + + /** + * Build the actual backup procedure runner that will do all the 'hard' work + */ + public class BackupSubprocedureBuilder implements SubprocedureFactory { + @Override + public Subprocedure buildSubprocedure(String name, byte[] data) { + return LogRollRegionServerProcedureManager.this.buildSubprocedure(data); + } + } + + @Override + public void initialize(RegionServerServices rss) throws KeeperException { + this.rss = rss; + if (!BackupManager.isBackupEnabled(rss.getConfiguration())) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + ProcedureCoordinationManager coordManager = new ZKProcedureCoordinationManager(rss); + this.memberRpcs = coordManager + .getProcedureMemberRpcs(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE); + + // read in the backup handler configuration properties + Configuration conf = rss.getConfiguration(); + long keepAlive = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT); + int opThreads = conf.getInt(BACKUP_REQUEST_THREADS_KEY, BACKUP_REQUEST_THREADS_DEFAULT); + // create the actual cohort member + ThreadPoolExecutor pool = + ProcedureMember.defaultPool(rss.getServerName().toString(), opThreads, keepAlive); + this.member = new ProcedureMember(memberRpcs, pool, new BackupSubprocedureBuilder()); + } + + @Override + public String getProcedureSignature() { + return "backup-proc"; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java new file mode 100644 index 000000000000..4228000d1966 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java @@ -0,0 +1,57 @@ +/* + * 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.util; + +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hbase.TableName; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Backup set is a named group of HBase tables, which are managed together by Backup/Restore + * framework. Instead of using list of tables in backup or restore operation, one can use set's name + * instead. + */ +@InterfaceAudience.Private +public class BackupSet { + private final String name; + private final List tables; + + public BackupSet(String name, List tables) { + this.name = name; + this.tables = tables; + } + + public String getName() { + return name; + } + + public List getTables() { + return tables; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(name).append("={"); + sb.append(StringUtils.join(tables, ',')); + sb.append("}"); + return sb.toString(); + } + +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java new file mode 100644 index 000000000000..164ae7e73765 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -0,0 +1,742 @@ +/* + * 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.util; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; +import java.util.TreeSet; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterators; + +/** + * A collection for methods used by multiple classes to backup HBase tables. + */ +@InterfaceAudience.Private +public final class BackupUtils { + private static final Logger LOG = LoggerFactory.getLogger(BackupUtils.class); + public static final String LOGNAME_SEPARATOR = "."; + public static final int MILLISEC_IN_HOUR = 3600000; + + private BackupUtils() { + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp value + * for the RS among the tables. + * @param rsLogTimestampMap timestamp map + * @return the min timestamp of each RS + */ + public static Map + getRSLogTimestampMins(Map> rsLogTimestampMap) { + if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) { + return null; + } + + HashMap rsLogTimestampMins = new HashMap<>(); + HashMap> rsLogTimestampMapByRS = new HashMap<>(); + + for (Entry> tableEntry : rsLogTimestampMap.entrySet()) { + TableName table = tableEntry.getKey(); + Map rsLogTimestamp = tableEntry.getValue(); + for (Entry rsEntry : rsLogTimestamp.entrySet()) { + String rs = rsEntry.getKey(); + Long ts = rsEntry.getValue(); + rsLogTimestampMapByRS.putIfAbsent(rs, new HashMap<>()); + rsLogTimestampMapByRS.get(rs).put(table, ts); + } + } + + for (Entry> entry : rsLogTimestampMapByRS.entrySet()) { + String rs = entry.getKey(); + rsLogTimestampMins.put(rs, BackupUtils.getMinValue(entry.getValue())); + } + + return rsLogTimestampMins; + } + + /** + * copy out Table RegionInfo into incremental backup image need to consider move this logic into + * HBackupFileSystem + * @param conn connection + * @param backupInfo backup info + * @param conf configuration + * @throws IOException exception + */ + public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo, Configuration conf) + throws IOException { + Path rootDir = CommonFSUtils.getRootDir(conf); + FileSystem fs = rootDir.getFileSystem(conf); + + // for each table in the table set, copy out the table info and region + // info files in the correct directory structure + try (Admin admin = conn.getAdmin()) { + for (TableName table : backupInfo.getTables()) { + if (!admin.tableExists(table)) { + LOG.warn("Table " + table + " does not exists, skipping it."); + continue; + } + TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table); + + // write a copy of descriptor to the target directory + Path target = new Path(backupInfo.getTableBackupDir(table)); + FileSystem targetFs = target.getFileSystem(conf); + FSTableDescriptors descriptors = + new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf)); + descriptors.createTableDescriptorForTableDirectory(target, orig, false); + LOG.debug("Attempting to copy table info for:" + table + " target: " + target + + " descriptor: " + orig); + LOG.debug("Finished copying tableinfo."); + List regions = MetaTableAccessor.getTableRegions(conn, table); + // For each region, write the region info to disk + LOG.debug("Starting to write region info for table " + table); + for (RegionInfo regionInfo : regions) { + Path regionDir = FSUtils + .getRegionDirFromTableDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo); + regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName()); + writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo); + } + LOG.debug("Finished writing region info for table " + table); + } + } + } + + /** + * Write the .regioninfo file on-disk. + */ + public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs, + final Path regionInfoDir, RegionInfo regionInfo) throws IOException { + final byte[] content = RegionInfo.toDelimitedByteArray(regionInfo); + Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR); + // First check to get the permissions + FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY); + // Write the RegionInfo file content + FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null); + try { + out.write(content); + } finally { + out.close(); + } + } + + /** + * Parses hostname:port from WAL file path + * @param p path to WAL file + * @return hostname:port + */ + public static String parseHostNameFromLogFile(Path p) { + try { + if (AbstractFSWALProvider.isArchivedLogFile(p)) { + return BackupUtils.parseHostFromOldLog(p); + } else { + ServerName sname = AbstractFSWALProvider.getServerNameFromWALDirectoryName(p); + if (sname != null) { + return sname.getAddress().toString(); + } else { + LOG.error("Skip log file (can't parse): " + p); + return null; + } + } + } catch (Exception e) { + LOG.error("Skip log file (can't parse): " + p, e); + return null; + } + } + + /** + * Returns WAL file name + * @param walFileName WAL file name + * @return WAL file name + */ + public static String getUniqueWALFileNamePart(String walFileName) { + return getUniqueWALFileNamePart(new Path(walFileName)); + } + + /** + * Returns WAL file name + * @param p WAL file path + * @return WAL file name + */ + public static String getUniqueWALFileNamePart(Path p) { + return p.getName(); + } + + /** + * Get the total length of files under the given directory recursively. + * @param fs The hadoop file system + * @param dir The target directory + * @return the total length of files + * @throws IOException exception + */ + public static long getFilesLength(FileSystem fs, Path dir) throws IOException { + long totalLength = 0; + FileStatus[] files = CommonFSUtils.listStatus(fs, dir); + if (files != null) { + for (FileStatus fileStatus : files) { + if (fileStatus.isDirectory()) { + totalLength += getFilesLength(fs, fileStatus.getPath()); + } else { + totalLength += fileStatus.getLen(); + } + } + } + return totalLength; + } + + /** + * Get list of all old WAL files (WALs and archive) + * @param c configuration + * @param hostTimestampMap {host,timestamp} map + * @return list of WAL files + * @throws IOException exception + */ + public static List getWALFilesOlderThan(final Configuration c, + final HashMap hostTimestampMap) throws IOException { + Path walRootDir = CommonFSUtils.getWALRootDir(c); + Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); + List logFiles = new ArrayList<>(); + + PathFilter filter = p -> { + try { + if (AbstractFSWALProvider.isMetaFile(p)) { + return false; + } + String host = parseHostNameFromLogFile(p); + if (host == null) { + return false; + } + Long oldTimestamp = hostTimestampMap.get(host); + Long currentLogTS = BackupUtils.getCreationTime(p); + return currentLogTS <= oldTimestamp; + } catch (Exception e) { + LOG.warn("Can not parse" + p, e); + return false; + } + }; + FileSystem walFs = CommonFSUtils.getWALFileSystem(c); + logFiles = BackupUtils.getFiles(walFs, logDir, logFiles, filter); + logFiles = BackupUtils.getFiles(walFs, oldLogDir, logFiles, filter); + return logFiles; + } + + public static TableName[] parseTableNames(String tables) { + if (tables == null) { + return null; + } + return Splitter.on(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND).splitToStream(tables) + .map(TableName::valueOf).toArray(TableName[]::new); + } + + /** + * Check whether the backup path exist + * @param backupStr backup + * @param conf configuration + * @return Yes if path exists + * @throws IOException exception + */ + public static boolean checkPathExist(String backupStr, Configuration conf) throws IOException { + boolean isExist = false; + Path backupPath = new Path(backupStr); + FileSystem fileSys = backupPath.getFileSystem(conf); + String targetFsScheme = fileSys.getUri().getScheme(); + if (LOG.isTraceEnabled()) { + LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme); + } + if (fileSys.exists(backupPath)) { + isExist = true; + } + return isExist; + } + + /** + * Check target path first, confirm it doesn't exist before backup + * @param backupRootPath backup destination path + * @param conf configuration + * @throws IOException exception + */ + public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException { + boolean targetExists; + try { + targetExists = checkPathExist(backupRootPath, conf); + } catch (IOException e) { + String expMsg = e.getMessage(); + String newMsg = null; + if (expMsg.contains("No FileSystem for scheme")) { + newMsg = + "Unsupported filesystem scheme found in the backup target url. Error Message: " + expMsg; + LOG.error(newMsg); + throw new IOException(newMsg); + } else { + throw e; + } + } + + if (targetExists) { + LOG.info("Using existing backup root dir: " + backupRootPath); + } else { + LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created."); + } + } + + /** + * Get the min value for all the Values a map. + * @param map map + * @return the min value + */ + public static Long getMinValue(Map map) { + Long minTimestamp = null; + if (map != null) { + ArrayList timestampList = new ArrayList<>(map.values()); + Collections.sort(timestampList); + // The min among all the RS log timestamps will be kept in backup system table table. + minTimestamp = timestampList.get(0); + } + return minTimestamp; + } + + /** + * Parses host name:port from archived WAL path + * @param p path + * @return host name + */ + public static String parseHostFromOldLog(Path p) { + try { + String n = p.getName(); + int idx = n.lastIndexOf(LOGNAME_SEPARATOR); + String s = URLDecoder.decode(n.substring(0, idx), "UTF8"); + return ServerName.valueOf(s).getAddress().toString(); + } catch (Exception e) { + LOG.warn("Skip log file (can't parse): {}", p); + return null; + } + } + + /** + * Given the log file, parse the timestamp from the file name. The timestamp is the last number. + * @param p a path to the log file + * @return the timestamp + * @throws IOException exception + */ + public static Long getCreationTime(Path p) throws IOException { + int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR); + if (idx < 0) { + throw new IOException("Cannot parse timestamp from path " + p); + } + String ts = p.getName().substring(idx + 1); + return Long.parseLong(ts); + } + + public static List getFiles(FileSystem fs, Path rootDir, List files, + PathFilter filter) throws IOException { + RemoteIterator it = fs.listFiles(rootDir, true); + + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (lfs.isDirectory()) { + continue; + } + // apply filter + if (filter.accept(lfs.getPath())) { + files.add(lfs.getPath().toString()); + } + } + return files; + } + + public static void cleanupBackupData(BackupInfo context, Configuration conf) throws IOException { + cleanupHLogDir(context, conf); + cleanupTargetDir(context, conf); + } + + /** + * Clean up directories which are generated when DistCp copying hlogs + * @param backupInfo backup info + * @param conf configuration + * @throws IOException exception + */ + private static void cleanupHLogDir(BackupInfo backupInfo, Configuration conf) throws IOException { + String logDir = backupInfo.getHLogTargetDir(); + if (logDir == null) { + LOG.warn("No log directory specified for " + backupInfo.getBackupId()); + return; + } + + Path rootPath = new Path(logDir).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = listStatus(fs, rootPath, null); + if (files == null) { + return; + } + for (FileStatus file : files) { + LOG.debug("Delete log files: " + file.getPath().getName()); + fs.delete(file.getPath(), true); + } + } + + private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) { + try { + // clean up the data at target directory + LOG.debug("Trying to cleanup up target dir : " + backupInfo.getBackupId()); + String targetDir = backupInfo.getBackupRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupInfo.getBackupId()); + return; + } + + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + for (TableName table : backupInfo.getTables()) { + Path targetDirPath = new Path( + getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = listStatus(outputFs, tableDir, null); + if (backups == null || backups.length == 0) { + outputFs.delete(tableDir, true); + LOG.debug(tableDir.toString() + " is empty, remove it."); + } + } + outputFs.delete(new Path(targetDir, backupInfo.getBackupId()), true); + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " at " + + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Given the backup root dir, backup id and the table name, return the backup image location, + * which is also where the backup manifest file is. return value look like: + * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/" + * @param backupRootDir backup root directory + * @param backupId backup id + * @param tableName table name + * @return backupPath String for the particular table + */ + public static String getTableBackupDir(String backupRootDir, String backupId, + TableName tableName) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString() + + Path.SEPARATOR; + } + + /** + * Sort history list by start time in descending order. + * @param historyList history list + * @return sorted list of BackupCompleteData + */ + public static ArrayList sortHistoryListDesc(ArrayList historyList) { + ArrayList list = new ArrayList<>(); + TreeMap map = new TreeMap<>(); + for (BackupInfo h : historyList) { + map.put(Long.toString(h.getStartTs()), h); + } + Iterator i = map.descendingKeySet().iterator(); + while (i.hasNext()) { + list.add(map.get(i.next())); + } + return list; + } + + /** + * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates + * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and + * return an empty FileStatus[] while Hadoop 2 will throw FileNotFoundException. + * @param fs file system + * @param dir directory + * @param filter path filter + * @return null if dir is empty or doesn't exist, otherwise FileStatus array + */ + public static FileStatus[] listStatus(final FileSystem fs, final Path dir, + final PathFilter filter) throws IOException { + FileStatus[] status = null; + try { + status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter); + } catch (FileNotFoundException fnfe) { + // if directory doesn't exist, return null + if (LOG.isTraceEnabled()) { + LOG.trace(dir + " doesn't exist"); + } + } + + if (status == null || status.length < 1) { + return null; + } + + return status; + } + + /** + * Return the 'path' component of a Path. In Hadoop, Path is a URI. This method returns the 'path' + * component of a Path's URI: e.g. If a Path is + * hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir, this method returns + * /hbase_trunk/TestTable/compaction.dir. This method is useful if you want to print + * out a Path without qualifying Filesystem instance. + * @param p file system Path whose 'path' component we are to return. + * @return Path portion of the Filesystem + */ + public static String getPath(Path p) { + return p.toUri().getPath(); + } + + /** + * Given the backup root dir and the backup id, return the log file location for an incremental + * backup. + * @param backupRootDir backup root directory + * @param backupId backup id + * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738" + */ + public static String getLogBackupDir(String backupRootDir, String backupId) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + HConstants.HREGION_LOGDIR_NAME; + } + + private static List getHistory(Configuration conf, Path backupRootPath) + throws IOException { + // Get all (n) history from backup root destination + + FileSystem fs = FileSystem.get(backupRootPath.toUri(), conf); + RemoteIterator it = fs.listLocatedStatus(backupRootPath); + + List infos = new ArrayList<>(); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + + if (!lfs.isDirectory()) { + continue; + } + + String backupId = lfs.getPath().getName(); + try { + BackupInfo info = loadBackupInfo(backupRootPath, backupId, fs); + infos.add(info); + } catch (IOException e) { + LOG.error("Can not load backup info from: " + lfs.getPath(), e); + } + } + // Sort + Collections.sort(infos, new Comparator() { + @Override + public int compare(BackupInfo o1, BackupInfo o2) { + long ts1 = getTimestamp(o1.getBackupId()); + long ts2 = getTimestamp(o2.getBackupId()); + + if (ts1 == ts2) { + return 0; + } + + return ts1 < ts2 ? 1 : -1; + } + + private long getTimestamp(String backupId) { + return Long.parseLong(Iterators.get(Splitter.on('_').split(backupId).iterator(), 1)); + } + }); + return infos; + } + + public static List getHistory(Configuration conf, int n, Path backupRootPath, + BackupInfo.Filter... filters) throws IOException { + List infos = getHistory(conf, backupRootPath); + List ret = new ArrayList<>(); + for (BackupInfo info : infos) { + if (ret.size() == n) { + break; + } + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(info)) { + passed = false; + break; + } + } + if (passed) { + ret.add(info); + } + } + return ret; + } + + public static BackupInfo loadBackupInfo(Path backupRootPath, String backupId, FileSystem fs) + throws IOException { + Path backupPath = new Path(backupRootPath, backupId); + + RemoteIterator it = fs.listFiles(backupPath, true); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (lfs.getPath().getName().equals(BackupManifest.MANIFEST_FILE_NAME)) { + // Load BackupManifest + BackupManifest manifest = new BackupManifest(fs, lfs.getPath().getParent()); + BackupInfo info = manifest.toBackupInfo(); + return info; + } + } + return null; + } + + /** + * Create restore request. + * @param backupRootDir backup root dir + * @param backupId backup id + * @param check check only + * @param fromTables table list from + * @param toTables table list to + * @param isOverwrite overwrite data + * @return request obkect + */ + public static RestoreRequest createRestoreRequest(String backupRootDir, String backupId, + boolean check, TableName[] fromTables, TableName[] toTables, boolean isOverwrite) { + RestoreRequest.Builder builder = new RestoreRequest.Builder(); + RestoreRequest request = + builder.withBackupRootDir(backupRootDir).withBackupId(backupId).withCheck(check) + .withFromTables(fromTables).withToTables(toTables).withOvewrite(isOverwrite).build(); + return request; + } + + public static boolean validate(HashMap backupManifestMap, + Configuration conf) throws IOException { + boolean isValid = true; + + for (Entry manifestEntry : backupManifestMap.entrySet()) { + TableName table = manifestEntry.getKey(); + TreeSet imageSet = new TreeSet<>(); + + ArrayList depList = manifestEntry.getValue().getDependentListByTable(table); + if (depList != null && !depList.isEmpty()) { + imageSet.addAll(depList); + } + + LOG.info("Dependent image(s) from old to new:"); + for (BackupImage image : imageSet) { + String imageDir = + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table); + if (!BackupUtils.checkPathExist(imageDir, conf)) { + LOG.error("ERROR: backup image does not exist: " + imageDir); + isValid = false; + break; + } + LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available"); + } + } + return isValid; + } + + public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit) + throws IOException { + FileSystem fs = FileSystem.get(conf); + String tmp = + conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, fs.getHomeDirectory() + "/hbase-staging"); + Path path = new Path(tmp + Path.SEPARATOR + "bulk_output-" + tableName + "-" + + EnvironmentEdgeManager.currentTime()); + if (deleteOnExit) { + fs.deleteOnExit(path); + } + return path; + } + + public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException { + return getBulkOutputDir(tableName, conf, true); + } + + public static String getFileNameCompatibleString(TableName table) { + return table.getNamespaceAsString() + "-" + table.getQualifierAsString(); + } + + public static boolean failed(int result) { + return result != 0; + } + + public static boolean succeeded(int result) { + return result == 0; + } + + public static BulkLoadHFiles createLoader(Configuration config) { + // set configuration for restore: + // LoadIncrementalHFile needs more time + // hbase.rpc.timeout 600000 + // calculates + Configuration conf = new Configuration(config); + conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR); + + // By default, it is 32 and loader will fail if # of files in any region exceed this + // limit. Bad for snapshot restore. + conf.setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE); + conf.set(BulkLoadHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes"); + return BulkLoadHFiles.create(conf); + } + + public static String findMostRecentBackupId(String[] backupIds) { + long recentTimestamp = Long.MIN_VALUE; + for (String backupId : backupIds) { + long ts = Long.parseLong(Iterators.get(Splitter.on('_').split(backupId).iterator(), 1)); + if (ts > recentTimestamp) { + recentTimestamp = ts; + } + } + return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp; + } + +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java new file mode 100644 index 000000000000..bf2aa14046db --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java @@ -0,0 +1,522 @@ +/* + * 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.util; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.TreeMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.NamespaceNotFoundException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreJob; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; + +/** + * A collection for methods used by multiple classes to restore HBase tables. + */ +@InterfaceAudience.Private +public class RestoreTool { + public static final Logger LOG = LoggerFactory.getLogger(BackupUtils.class); + private final static long TABLE_AVAILABILITY_WAIT_TIME = 180000; + + private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR }; + protected Configuration conf; + protected Path backupRootPath; + protected String backupId; + protected FileSystem fs; + + // store table name and snapshot dir mapping + private final HashMap snapshotMap = new HashMap<>(); + + public RestoreTool(Configuration conf, final Path backupRootPath, final String backupId) + throws IOException { + this.conf = conf; + this.backupRootPath = backupRootPath; + this.backupId = backupId; + this.fs = backupRootPath.getFileSystem(conf); + } + + /** + * return value represent path for: + * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn" + * @param tableName table name + * @return path to table archive + * @throws IOException exception + */ + Path getTableArchivePath(TableName tableName) throws IOException { + Path baseDir = + new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), + HConstants.HFILE_ARCHIVE_DIRECTORY); + Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR); + Path archivePath = new Path(dataDir, tableName.getNamespaceAsString()); + Path tableArchivePath = new Path(archivePath, tableName.getQualifierAsString()); + if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) { + LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists"); + tableArchivePath = null; // empty table has no archive + } + return tableArchivePath; + } + + /** + * Gets region list + * @param tableName table name + * @return RegionList region list + * @throws IOException exception + */ + ArrayList getRegionList(TableName tableName) throws IOException { + Path tableArchivePath = getTableArchivePath(tableName); + ArrayList regionDirList = new ArrayList<>(); + FileStatus[] children = fs.listStatus(tableArchivePath); + for (FileStatus childStatus : children) { + // here child refer to each region(Name) + Path child = childStatus.getPath(); + regionDirList.add(child); + } + return regionDirList; + } + + void modifyTableSync(Connection conn, TableDescriptor desc) throws IOException { + try (Admin admin = conn.getAdmin()) { + admin.modifyTable(desc); + int attempt = 0; + int maxAttempts = 600; + while (!admin.isTableAvailable(desc.getTableName())) { + Thread.sleep(100); + attempt++; + if (attempt++ > maxAttempts) { + throw new IOException("Timeout expired " + (maxAttempts * 100) + "ms"); + } + } + } catch (Exception e) { + throw new IOException(e); + } + } + + /** + * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently + * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in + * the future + * @param conn HBase connection + * @param tableBackupPath backup path + * @param logDirs : incremental backup folders, which contains WAL + * @param tableNames : source tableNames(table names were backuped) + * @param newTableNames : target tableNames(table names to be restored to) + * @param incrBackupId incremental backup Id + * @throws IOException exception + */ + public void incrementalRestoreTable(Connection conn, Path tableBackupPath, Path[] logDirs, + TableName[] tableNames, TableName[] newTableNames, String incrBackupId) throws IOException { + try (Admin admin = conn.getAdmin()) { + if (tableNames.length != newTableNames.length) { + throw new IOException("Number of source tables and target tables does not match!"); + } + FileSystem fileSys = tableBackupPath.getFileSystem(this.conf); + + // for incremental backup image, expect the table already created either by user or previous + // full backup. Here, check that all new tables exists + for (TableName tableName : newTableNames) { + if (!admin.tableExists(tableName)) { + throw new IOException("HBase table " + tableName + + " does not exist. Create the table first, e.g. by restoring a full backup."); + } + } + // adjust table schema + for (int i = 0; i < tableNames.length; i++) { + TableName tableName = tableNames[i]; + TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId); + if (tableDescriptor == null) { + throw new IOException("Can't find " + tableName + "'s descriptor."); + } + LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId); + + TableName newTableName = newTableNames[i]; + TableDescriptor newTableDescriptor = admin.getDescriptor(newTableName); + List families = Arrays.asList(tableDescriptor.getColumnFamilies()); + List existingFamilies = + Arrays.asList(newTableDescriptor.getColumnFamilies()); + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(newTableDescriptor); + boolean schemaChangeNeeded = false; + for (ColumnFamilyDescriptor family : families) { + if (!existingFamilies.contains(family)) { + builder.setColumnFamily(family); + schemaChangeNeeded = true; + } + } + for (ColumnFamilyDescriptor family : existingFamilies) { + if (!families.contains(family)) { + builder.removeColumnFamily(family.getName()); + schemaChangeNeeded = true; + } + } + if (schemaChangeNeeded) { + modifyTableSync(conn, builder.build()); + LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor); + } + } + RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf); + + restoreService.run(logDirs, tableNames, newTableNames, false); + } + } + + public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName, + TableName newTableName, boolean truncateIfExists, String lastIncrBackupId) throws IOException { + createAndRestoreTable(conn, tableName, newTableName, tableBackupPath, truncateIfExists, + lastIncrBackupId); + } + + /** + * Returns value represent path for path to backup table snapshot directory: + * "/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot" + * @param backupRootPath backup root path + * @param tableName table name + * @param backupId backup Id + * @return path for snapshot + */ + Path getTableSnapshotPath(Path backupRootPath, TableName tableName, String backupId) { + return new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), + HConstants.SNAPSHOT_DIR_NAME); + } + + /** + * Returns value represent path for: + * ""/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot/ + * snapshot_1396650097621_namespace_table" this path contains .snapshotinfo, .tabledesc (0.96 and + * 0.98) this path contains .snapshotinfo, .data.manifest (trunk) + * @param tableName table name + * @return path to table info + * @throws IOException exception + */ + Path getTableInfoPath(TableName tableName) throws IOException { + Path tableSnapShotPath = getTableSnapshotPath(backupRootPath, tableName, backupId); + Path tableInfoPath = null; + + // can't build the path directly as the timestamp values are different + FileStatus[] snapshots = fs.listStatus(tableSnapShotPath, + new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs)); + for (FileStatus snapshot : snapshots) { + tableInfoPath = snapshot.getPath(); + // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest"; + if (tableInfoPath.getName().endsWith("data.manifest")) { + break; + } + } + return tableInfoPath; + } + + /** + * Get table descriptor + * @param tableName is the table backed up + * @return {@link TableDescriptor} saved in backup image of the table + */ + TableDescriptor getTableDesc(TableName tableName) throws IOException { + Path tableInfoPath = this.getTableInfoPath(tableName); + SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath); + SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc); + TableDescriptor tableDescriptor = manifest.getTableDescriptor(); + if (!tableDescriptor.getTableName().equals(tableName)) { + LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: " + + tableInfoPath.toString()); + LOG.error( + "tableDescriptor.getNameAsString() = " + tableDescriptor.getTableName().getNameAsString()); + throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName + + " under tableInfoPath: " + tableInfoPath.toString()); + } + return tableDescriptor; + } + + private TableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName, + String lastIncrBackupId) throws IOException { + if (lastIncrBackupId != null) { + String target = + BackupUtils.getTableBackupDir(backupRootPath.toString(), lastIncrBackupId, tableName); + return FSTableDescriptors.getTableDescriptorFromFs(fileSys, new Path(target)); + } + return null; + } + + private void createAndRestoreTable(Connection conn, TableName tableName, TableName newTableName, + Path tableBackupPath, boolean truncateIfExists, String lastIncrBackupId) throws IOException { + if (newTableName == null) { + newTableName = tableName; + } + FileSystem fileSys = tableBackupPath.getFileSystem(this.conf); + + // get table descriptor first + TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId); + if (tableDescriptor != null) { + LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId); + } + + if (tableDescriptor == null) { + Path tableSnapshotPath = getTableSnapshotPath(backupRootPath, tableName, backupId); + if (fileSys.exists(tableSnapshotPath)) { + // snapshot path exist means the backup path is in HDFS + // check whether snapshot dir already recorded for target table + if (snapshotMap.get(tableName) != null) { + SnapshotDescription desc = + SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath); + SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc); + tableDescriptor = manifest.getTableDescriptor(); + } else { + tableDescriptor = getTableDesc(tableName); + snapshotMap.put(tableName, getTableInfoPath(tableName)); + } + if (tableDescriptor == null) { + LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost"); + } + } else { + throw new IOException( + "Table snapshot directory: " + tableSnapshotPath + " does not exist."); + } + } + + Path tableArchivePath = getTableArchivePath(tableName); + if (tableArchivePath == null) { + if (tableDescriptor != null) { + // find table descriptor but no archive dir means the table is empty, create table and exit + if (LOG.isDebugEnabled()) { + LOG.debug("find table descriptor but no archive dir for table " + tableName + + ", will only create table"); + } + tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor); + checkAndCreateTable(conn, newTableName, null, tableDescriptor, truncateIfExists); + return; + } else { + throw new IllegalStateException( + "Cannot restore hbase table because directory '" + " tableArchivePath is null."); + } + } + + if (tableDescriptor == null) { + tableDescriptor = TableDescriptorBuilder.newBuilder(newTableName).build(); + } else { + tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor); + } + + // record all region dirs: + // load all files in dir + try { + ArrayList regionPathList = getRegionList(tableName); + + // should only try to create the table with all region informations, so we could pre-split + // the regions in fine grain + checkAndCreateTable(conn, newTableName, regionPathList, tableDescriptor, truncateIfExists); + RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf); + Path[] paths = new Path[regionPathList.size()]; + regionPathList.toArray(paths); + restoreService.run(paths, new TableName[] { tableName }, new TableName[] { newTableName }, + true); + + } catch (Exception e) { + LOG.error(e.toString(), e); + throw new IllegalStateException("Cannot restore hbase table", e); + } + } + + /** + * Gets region list + * @param tableArchivePath table archive path + * @return RegionList region list + * @throws IOException exception + */ + ArrayList getRegionList(Path tableArchivePath) throws IOException { + ArrayList regionDirList = new ArrayList<>(); + FileStatus[] children = fs.listStatus(tableArchivePath); + for (FileStatus childStatus : children) { + // here child refer to each region(Name) + Path child = childStatus.getPath(); + regionDirList.add(child); + } + return regionDirList; + } + + /** + * Calculate region boundaries and add all the column families to the table descriptor + * @param regionDirList region dir list + * @return a set of keys to store the boundaries + */ + byte[][] generateBoundaryKeys(ArrayList regionDirList) throws IOException { + TreeMap map = new TreeMap<>(Bytes.BYTES_COMPARATOR); + // Build a set of keys to store the boundaries + // calculate region boundaries and add all the column families to the table descriptor + for (Path regionDir : regionDirList) { + LOG.debug("Parsing region dir: " + regionDir); + Path hfofDir = regionDir; + + if (!fs.exists(hfofDir)) { + LOG.warn("HFileOutputFormat dir " + hfofDir + " not found"); + } + + FileStatus[] familyDirStatuses = fs.listStatus(hfofDir); + if (familyDirStatuses == null) { + throw new IOException("No families found in " + hfofDir); + } + + for (FileStatus stat : familyDirStatuses) { + if (!stat.isDirectory()) { + LOG.warn("Skipping non-directory " + stat.getPath()); + continue; + } + boolean isIgnore = false; + String pathName = stat.getPath().getName(); + for (String ignore : ignoreDirs) { + if (pathName.contains(ignore)) { + LOG.warn("Skipping non-family directory" + pathName); + isIgnore = true; + break; + } + } + if (isIgnore) { + continue; + } + Path familyDir = stat.getPath(); + LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]"); + // Skip _logs, etc + if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) { + continue; + } + + // start to parse hfile inside one family dir + Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir)); + for (Path hfile : hfiles) { + if ( + hfile.getName().startsWith("_") || hfile.getName().startsWith(".") + || StoreFileInfo.isReference(hfile.getName()) + || HFileLink.isHFileLink(hfile.getName()) + ) { + continue; + } + HFile.Reader reader = HFile.createReader(fs, hfile, conf); + final byte[] first, last; + try { + first = reader.getFirstRowKey().get(); + last = reader.getLastRowKey().get(); + LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first=" + + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last)); + + // To eventually infer start key-end key boundaries + Integer value = map.containsKey(first) ? (Integer) map.get(first) : 0; + map.put(first, value + 1); + value = map.containsKey(last) ? (Integer) map.get(last) : 0; + map.put(last, value - 1); + } finally { + reader.close(); + } + } + } + } + return BulkLoadHFilesTool.inferBoundaries(map); + } + + /** + * Prepare the table for bulkload, most codes copied from {@code createTable} method in + * {@code BulkLoadHFilesTool}. + * @param conn connection + * @param targetTableName target table name + * @param regionDirList region directory list + * @param htd table descriptor + * @param truncateIfExists truncates table if exists + * @throws IOException exception + */ + private void checkAndCreateTable(Connection conn, TableName targetTableName, + ArrayList regionDirList, TableDescriptor htd, boolean truncateIfExists) + throws IOException { + try (Admin admin = conn.getAdmin()) { + boolean createNew = false; + if (admin.tableExists(targetTableName)) { + if (truncateIfExists) { + LOG.info( + "Truncating exising target table '" + targetTableName + "', preserving region splits"); + admin.disableTable(targetTableName); + admin.truncateTable(targetTableName, true); + } else { + LOG.info("Using exising target table '" + targetTableName + "'"); + } + } else { + createNew = true; + } + if (createNew) { + LOG.info("Creating target table '" + targetTableName + "'"); + byte[][] keys = null; + try { + if (regionDirList == null || regionDirList.size() == 0) { + admin.createTable(htd); + } else { + keys = generateBoundaryKeys(regionDirList); + // create table using table descriptor and region boundaries + admin.createTable(htd, keys); + } + } catch (NamespaceNotFoundException e) { + LOG.warn("There was no namespace and the same will be created"); + String namespaceAsString = targetTableName.getNamespaceAsString(); + LOG.info("Creating target namespace '" + namespaceAsString + "'"); + admin.createNamespace(NamespaceDescriptor.create(namespaceAsString).build()); + if (null == keys) { + admin.createTable(htd); + } else { + admin.createTable(htd, keys); + } + } + + } + long startTime = EnvironmentEdgeManager.currentTime(); + while (!admin.isTableAvailable(targetTableName)) { + try { + Thread.sleep(100); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) { + throw new IOException("Time out " + TABLE_AVAILABILITY_WAIT_TIME + "ms expired, table " + + targetTableName + " is still not available"); + } + } + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java new file mode 100644 index 000000000000..1380de2a3f3a --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -0,0 +1,509 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +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.impl.FullTableBackupClient; +import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager; +import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.master.cleaner.LogCleaner; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner; +import org.apache.hadoop.hbase.security.HadoopSecurityEnabledUserProviderForTesting; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.access.SecureTestUtil; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is only a base for other integration-level backup tests. Do not add tests here. + * TestBackupSmallTests is where tests that don't require bring machines up/down should go All other + * tests should have their own classes and extend this one + */ +public class TestBackupBase { + private static final Logger LOG = LoggerFactory.getLogger(TestBackupBase.class); + + protected static HBaseTestingUtility TEST_UTIL; + protected static HBaseTestingUtility TEST_UTIL2; + protected static Configuration conf1; + protected static Configuration conf2; + + protected static TableName table1 = TableName.valueOf("table1"); + protected static TableDescriptor table1Desc; + protected static TableName table2 = TableName.valueOf("table2"); + protected static TableName table3 = TableName.valueOf("table3"); + protected static TableName table4 = TableName.valueOf("table4"); + + protected static TableName table1_restore = TableName.valueOf("default:table1"); + protected static TableName table2_restore = TableName.valueOf("ns2:table2"); + protected static TableName table3_restore = TableName.valueOf("ns3:table3_restore"); + + protected static final int NB_ROWS_IN_BATCH = 99; + protected static final byte[] qualName = Bytes.toBytes("q1"); + protected static final byte[] famName = Bytes.toBytes("f"); + + protected static String BACKUP_ROOT_DIR; + protected static String BACKUP_REMOTE_ROOT_DIR; + protected static String provider = "defaultProvider"; + protected static boolean secure = false; + + protected static boolean autoRestoreOnFailure; + protected static boolean useSecondCluster; + + static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient { + public IncrementalTableBackupClientForTest() { + } + + public IncrementalTableBackupClientForTest(Connection conn, String backupId, + BackupRequest request) throws IOException { + super(conn, backupId, request); + } + + @Override + public void execute() throws IOException { + // case INCREMENTAL_COPY: + try { + // case PREPARE_INCREMENTAL: + failStageIf(Stage.stage_0); + beginBackup(backupManager, backupInfo); + + failStageIf(Stage.stage_1); + backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); + LOG.debug("For incremental backup, current table set is " + + backupManager.getIncrementalBackupTableSet()); + newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap(); + // copy out the table and region info files for each table + BackupUtils.copyTableRegionInfo(conn, backupInfo, conf); + // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT + convertWALsToHFiles(); + incrementalCopyHFiles(new String[] { getBulkOutputDir().toString() }, + backupInfo.getBackupRootDir()); + failStageIf(Stage.stage_2); + + // 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 + // Set the previousTimestampMap which is before this current log roll to the manifest. + Map> previousTimestampMap = + backupManager.readLogTimestampMap(); + backupInfo.setIncrTimestampMap(previousTimestampMap); + + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + failStageIf(Stage.stage_3); + + Map> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + Long newStartCode = + BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + + handleBulkLoad(backupInfo.getTableNames()); + failStageIf(Stage.stage_4); + + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf); + + } catch (Exception e) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", + BackupType.INCREMENTAL, conf); + throw new IOException(e); + } + } + } + + static class FullTableBackupClientForTest extends FullTableBackupClient { + public FullTableBackupClientForTest() { + } + + public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request) + throws IOException { + super(conn, backupId, request); + } + + @Override + public void execute() throws IOException { + // Get the stage ID to fail on + try (Admin admin = conn.getAdmin()) { + // Begin BACKUP + beginBackup(backupManager, backupInfo); + failStageIf(Stage.stage_0); + String savedStartCode; + boolean firstBackup; + // do snapshot for full table backup + savedStartCode = backupManager.readBackupStartCode(); + firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; + if (firstBackup) { + // This is our first backup. Let's put some marker to system table so that we can hold the + // logs while we do the backup. + backupManager.writeBackupStartCode(0L); + } + failStageIf(Stage.stage_1); + // 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 ..."); + + Map props = new HashMap<>(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + failStageIf(Stage.stage_2); + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + + // SNAPSHOT_TABLES: + backupInfo.setPhase(BackupPhase.SNAPSHOT); + for (TableName tableName : tableList) { + String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString(); + + snapshotTable(admin, tableName, snapshotName); + backupInfo.setSnapshotName(tableName, snapshotName); + } + failStageIf(Stage.stage_3); + // SNAPSHOT_COPY: + // do snapshot copy + LOG.debug("snapshot copy for " + backupId); + snapshotCopy(backupInfo); + // Updates incremental backup table set + backupManager.addIncrementalBackupTableSet(backupInfo.getTables()); + + // 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 + backupInfo.setState(BackupState.COMPLETE); + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + Map> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + Long newStartCode = + BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + failStageIf(Stage.stage_4); + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf); + + } catch (Exception e) { + + if (autoRestoreOnFailure) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", + BackupType.FULL, conf); + } + throw new IOException(e); + } + } + } + + public static void setUpHelper() throws Exception { + BACKUP_ROOT_DIR = Path.SEPARATOR + "backupUT"; + BACKUP_REMOTE_ROOT_DIR = Path.SEPARATOR + "backupUT"; + + if (secure) { + // set the always on security provider + UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(), + HadoopSecurityEnabledUserProviderForTesting.class); + // setup configuration + SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration()); + } + conf1.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf1); + BackupManager.decorateRegionServerConfiguration(conf1); + conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); + // Set TTL for old WALs to 1 sec to enforce fast cleaning of an archived + // WAL files + conf1.setLong(TimeToLiveLogCleaner.TTL_CONF_KEY, 1000); + conf1.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, 1000); + + // Set MultiWAL (with 2 default WAL files per RS) + conf1.set(WALFactory.WAL_PROVIDER, provider); + TEST_UTIL.startMiniCluster(); + + if (useSecondCluster) { + conf2 = HBaseConfiguration.create(conf1); + conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2"); + TEST_UTIL2 = new HBaseTestingUtility(conf2); + TEST_UTIL2.setZkCluster(TEST_UTIL.getZkCluster()); + TEST_UTIL2.startMiniDFSCluster(3); + String root2 = TEST_UTIL2.getConfiguration().get("fs.defaultFS"); + Path p = new Path(new Path(root2), "/tmp/wal"); + CommonFSUtils.setWALRootDir(TEST_UTIL2.getConfiguration(), p); + TEST_UTIL2.startMiniCluster(); + } + conf1 = TEST_UTIL.getConfiguration(); + + TEST_UTIL.startMiniMapReduceCluster(); + BACKUP_ROOT_DIR = + new Path(new Path(TEST_UTIL.getConfiguration().get("fs.defaultFS")), BACKUP_ROOT_DIR) + .toString(); + LOG.info("ROOTDIR " + BACKUP_ROOT_DIR); + if (useSecondCluster) { + BACKUP_REMOTE_ROOT_DIR = new Path( + new Path(TEST_UTIL2.getConfiguration().get("fs.defaultFS")) + BACKUP_REMOTE_ROOT_DIR) + .toString(); + LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR); + } + createTables(); + populateFromMasterConfig(TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(), conf1); + } + + /** + * Setup Cluster with appropriate configurations before running tests. + * @throws Exception if starting the mini cluster or setting up the tables fails + */ + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + conf1 = TEST_UTIL.getConfiguration(); + autoRestoreOnFailure = true; + useSecondCluster = false; + setUpHelper(); + } + + private static void populateFromMasterConfig(Configuration masterConf, Configuration conf) { + Iterator> it = masterConf.iterator(); + while (it.hasNext()) { + Entry e = it.next(); + conf.set(e.getKey(), e.getValue()); + } + } + + @AfterClass + public static void tearDown() throws Exception { + try { + SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getAdmin()); + } catch (Exception e) { + } + SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); + if (useSecondCluster) { + TEST_UTIL2.shutdownMiniCluster(); + } + TEST_UTIL.shutdownMiniCluster(); + TEST_UTIL.shutdownMiniMapReduceCluster(); + autoRestoreOnFailure = true; + useSecondCluster = false; + } + + Table insertIntoTable(Connection conn, TableName table, byte[] family, int id, int numRows) + throws IOException { + Table t = conn.getTable(table); + Put p1; + for (int i = 0; i < numRows; i++) { + p1 = new Put(Bytes.toBytes("row-" + table + "-" + id + "-" + i)); + p1.addColumn(family, qualName, Bytes.toBytes("val" + i)); + t.put(p1); + } + return t; + } + + protected BackupRequest createBackupRequest(BackupType type, List tables, + String path) { + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = + builder.withBackupType(type).withTableList(tables).withTargetRootDir(path).build(); + return request; + } + + protected String backupTables(BackupType type, List tables, String path) + throws IOException { + Connection conn = null; + BackupAdmin badmin = null; + String backupId; + try { + conn = ConnectionFactory.createConnection(conf1); + badmin = new BackupAdminImpl(conn); + BackupRequest request = createBackupRequest(type, tables, path); + backupId = badmin.backupTables(request); + } finally { + if (badmin != null) { + badmin.close(); + } + if (conn != null) { + conn.close(); + } + } + return backupId; + } + + protected String fullTableBackup(List tables) throws IOException { + return backupTables(BackupType.FULL, tables, BACKUP_ROOT_DIR); + } + + protected String incrementalTableBackup(List tables) throws IOException { + return backupTables(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + } + + protected static void loadTable(Table table) throws Exception { + Put p; // 100 + 1 row to t1_syncup + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p = new Put(Bytes.toBytes("row" + i)); + p.setDurability(Durability.SKIP_WAL); + p.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + table.put(p); + } + } + + protected static void createTables() throws Exception { + long tid = EnvironmentEdgeManager.currentTime(); + table1 = TableName.valueOf("test-" + tid); + Admin ha = TEST_UTIL.getAdmin(); + + // Create namespaces + ha.createNamespace(NamespaceDescriptor.create("ns1").build()); + ha.createNamespace(NamespaceDescriptor.create("ns2").build()); + ha.createNamespace(NamespaceDescriptor.create("ns3").build()); + ha.createNamespace(NamespaceDescriptor.create("ns4").build()); + + TableDescriptor desc = TableDescriptorBuilder.newBuilder(table1) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build(); + ha.createTable(desc); + table1Desc = desc; + Connection conn = ConnectionFactory.createConnection(conf1); + Table table = conn.getTable(table1); + loadTable(table); + table.close(); + table2 = TableName.valueOf("ns2:test-" + tid + 1); + desc = TableDescriptorBuilder.newBuilder(table2) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build(); + ha.createTable(desc); + table = conn.getTable(table2); + loadTable(table); + table.close(); + table3 = TableName.valueOf("ns3:test-" + tid + 2); + table = TEST_UTIL.createTable(table3, famName); + table.close(); + table4 = TableName.valueOf("ns4:test-" + tid + 3); + table = TEST_UTIL.createTable(table4, famName); + table.close(); + ha.close(); + conn.close(); + } + + protected boolean checkSucceeded(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + + if (status == null) { + return false; + } + + return status.getState() == BackupState.COMPLETE; + } + + protected boolean checkFailed(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + + if (status == null) { + return false; + } + + return status.getState() == BackupState.FAILED; + } + + private BackupInfo getBackupInfo(String backupId) throws IOException { + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + BackupInfo status = table.readBackupInfo(backupId); + return status; + } + } + + protected BackupAdmin getBackupAdmin() throws IOException { + return new BackupAdminImpl(TEST_UTIL.getConnection()); + } + + /** + * Helper method + */ + protected List toList(String... args) { + List ret = new ArrayList<>(); + for (int i = 0; i < args.length; i++) { + ret.add(TableName.valueOf(args[i])); + } + return ret; + } + + protected List getListOfWALFiles(Configuration c) throws IOException { + Path logRoot = new Path(CommonFSUtils.getWALRootDir(c), HConstants.HREGION_LOGDIR_NAME); + FileSystem fs = logRoot.getFileSystem(c); + RemoteIterator it = fs.listFiles(logRoot, true); + List logFiles = new ArrayList(); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (lfs.isFile() && !AbstractFSWALProvider.isMetaFile(lfs.getPath())) { + logFiles.add(lfs); + LOG.info(Objects.toString(lfs)); + } + } + return logFiles; + } + + protected void dumpBackupDir() throws IOException { + // Dump Backup Dir + FileSystem fs = FileSystem.get(conf1); + RemoteIterator it = fs.listFiles(new Path(BACKUP_ROOT_DIR), true); + while (it.hasNext()) { + LOG.debug(Objects.toString(it.next().getPath())); + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java new file mode 100644 index 000000000000..b56a15d3a4a8 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java @@ -0,0 +1,98 @@ +/* + * 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 org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupBoundaryTests extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupBoundaryTests.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupBoundaryTests.class); + + /** + * Verify that full backup is created on a single empty table correctly. + * @throws Exception if doing the full backup fails + */ + @Test + public void testFullBackupSingleEmpty() throws Exception { + LOG.info("create full backup image on single table"); + List tables = Lists.newArrayList(table3); + LOG.info("Finished Backup " + fullTableBackup(tables)); + } + + /** + * Verify that full backup is created on multiple empty tables correctly. + * @throws Exception if doing the full backup fails + */ + @Test + public void testFullBackupMultipleEmpty() throws Exception { + LOG.info("create full backup image on mulitple empty tables"); + + List tables = Lists.newArrayList(table3, table4); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on a single table that does not exist. + * @throws Exception if doing the full backup fails + */ + @Test(expected = IOException.class) + public void testFullBackupSingleDNE() throws Exception { + LOG.info("test full backup fails on a single table that does not exist"); + List tables = toList("tabledne"); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on multiple tables that do not exist. + * @throws Exception if doing the full backup fails + */ + @Test(expected = IOException.class) + public void testFullBackupMultipleDNE() throws Exception { + LOG.info("test full backup fails on multiple tables that do not exist"); + List tables = toList("table1dne", "table2dne"); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on tableset containing real and fake tables. + * @throws Exception if doing the full backup fails + */ + @Test(expected = IOException.class) + public void testFullBackupMixExistAndDNE() throws Exception { + LOG.info("create full backup fails on tableset containing real and fake table"); + + List tables = toList(table1.getNameAsString(), "tabledne"); + fullTableBackup(tables); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java new file mode 100644 index 000000000000..77bc9bdb9b2f --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java @@ -0,0 +1,444 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBackupCommandLineTool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupCommandLineTool.class); + + private final static String USAGE_DESCRIBE = "Usage: hbase backup describe "; + private final static String USAGE_CREATE = "Usage: hbase backup create"; + private final static String USAGE_HISTORY = "Usage: hbase backup history"; + private final static String USAGE_BACKUP = "Usage: hbase backup"; + private final static String USAGE_DELETE = "Usage: hbase backup delete"; + private final static String USAGE_PROGRESS = "Usage: hbase backup progress"; + private final static String USAGE_SET = "Usage: hbase backup set"; + private final static String USAGE_RESTORE = "Usage: hbase restore"; + + Configuration conf; + + @Before + public void setUpBefore() throws Exception { + conf = HBaseConfiguration.create(); + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + } + + @Test + public void testBackupDriverDescribeHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "describe", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + } + + @Test + public void testBackupDriverCreateTopLevelBackupDest() throws Exception { + String[] args = new String[] { "create", "full", "hdfs://localhost:1020", "-t", "t1" }; + int result = ToolRunner.run(conf, new BackupDriver(), args); + // FAILED + assertEquals(1, result); + } + + @Test + public void testBackupDriverCreateHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + } + + @Test + public void testBackupDriverHistoryHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "history", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "history", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + } + + @Test + public void testBackupDriverDeleteHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "delete", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + } + + @Test + public void testBackupDriverProgressHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "progress", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "progress", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); + } + + @Test + public void testBackupDriverSetHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "set", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + } + + @Test + public void testBackupDriverHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + } + + @Test + public void testRestoreDriverHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-help" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "-h" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + } + + @Test + public void testBackupDriverUnrecognizedCommand() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "command" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "command" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + } + + @Test + public void testBackupDriverUnrecognizedOption() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "history", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + } + + @Test + public void testRestoreDriverUnrecognizedOption() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-xx" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + + } + + @Test + public void testBackupDriverCreateWrongArgNumber() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "22" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "22", "22", "22", "22", "22" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + } + + @Test + public void testBackupDriverDeleteWrongArgNumber() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "delete" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + } + + @Test + public void testBackupDriverHistoryWrongArgs() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "history", "-n", "xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + } + + @Test + public void testBackupDriverWrongBackupDestination() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "full", "clicks" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf("ERROR: invalid backup destination") >= 0); + + } + + @Test + public void testBackupDriverBackupSetAndList() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "full", "file:/localhost", "-t", "clicks", "-s", "s" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf("ERROR: You can specify either backup set or list") >= 0); + + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java new file mode 100644 index 000000000000..0c4d44d489d8 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java @@ -0,0 +1,161 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdge; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDelete extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDelete.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupDelete.class); + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testBackupDelete() throws Exception { + LOG.info("test backup delete on a single table with data"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + String[] backupIds = new String[] { backupId }; + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo info = table.readBackupInfo(backupId); + Path path = new Path(info.getBackupRootDir(), backupId); + FileSystem fs = FileSystem.get(path.toUri(), conf1); + assertTrue(fs.exists(path)); + int deleted = getBackupAdmin().deleteBackups(backupIds); + + assertTrue(!fs.exists(path)); + assertTrue(fs.exists(new Path(info.getBackupRootDir()))); + assertTrue(1 == deleted); + table.close(); + LOG.info("delete_backup"); + } + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testBackupDeleteCommand() throws Exception { + LOG.info("test backup delete on a single table with data: command-line"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "delete", "-l", backupId }; + // Run backup + + try { + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + } catch (Exception e) { + LOG.error("failed", e); + } + LOG.info("delete_backup"); + String output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf("Deleted 1 backups") >= 0); + } + + @Test + public void testBackupPurgeOldBackupsCommand() throws Exception { + LOG.info("test backup delete (purge old backups) on a single table with data: command-line"); + List tableList = Lists.newArrayList(table1); + EnvironmentEdgeManager.injectEdge(new EnvironmentEdge() { + // time - 2 days + @Override + public long currentTime() { + return System.currentTimeMillis() - 2 * 24 * 3600 * 1000; + } + }); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + + EnvironmentEdgeManager.reset(); + + LOG.info("backup complete"); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + // Purge all backups which are older than 3 days + // Must return 0 (no backups were purged) + String[] args = new String[] { "delete", "-k", "3" }; + // Run backup + + try { + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + } catch (Exception e) { + LOG.error("failed", e); + Assert.fail(e.getMessage()); + } + String output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf("Deleted 0 backups") >= 0); + + // Purge all backups which are older than 1 days + // Must return 1 deleted backup + args = new String[] { "delete", "-k", "1" }; + // Run backup + baos.reset(); + try { + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + } catch (Exception e) { + LOG.error("failed", e); + Assert.fail(e.getMessage()); + } + output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf("Deleted 1 backups") >= 0); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java new file mode 100644 index 000000000000..2798e1a16f0d --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java @@ -0,0 +1,81 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(MediumTests.class) +public class TestBackupDeleteRestore extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDeleteRestore.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupDeleteRestore.class); + + /** + * Verify that load data- backup - delete some data - restore works as expected - deleted data get + * restored. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testBackupDeleteRestore() throws Exception { + LOG.info("test full restore on a single table empty table"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + int numRows = TEST_UTIL.countRows(table1); + Admin hba = TEST_UTIL.getAdmin(); + // delete row + try (Table table = TEST_UTIL.getConnection().getTable(table1)) { + Delete delete = new Delete(Bytes.toBytes("row0")); + table.delete(delete); + hba.flush(table1); + } + + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = null;// new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap, true)); + + int numRowsAfterRestore = TEST_UTIL.countRows(table1); + assertEquals(numRows, numRowsAfterRestore); + hba.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java new file mode 100644 index 000000000000..bc1b110b672b --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java @@ -0,0 +1,199 @@ +/* + * 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.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; +import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.MasterObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +/** + * This class is only a base for other integration-level backup tests. Do not add tests here. + * TestBackupSmallTests is where tests that don't require bring machines up/down should go All other + * tests should have their own classes and extend this one + */ +@Category(LargeTests.class) +public class TestBackupDeleteWithFailures extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDeleteWithFailures.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupDeleteWithFailures.class); + + public enum Failure { + NO_FAILURES, + PRE_SNAPSHOT_FAILURE, + PRE_DELETE_SNAPSHOT_FAILURE, + POST_DELETE_SNAPSHOT_FAILURE + } + + public static class MasterSnapshotObserver implements MasterCoprocessor, MasterObserver { + List failures = new ArrayList<>(); + + public void setFailures(Failure... f) { + failures.clear(); + for (int i = 0; i < f.length; i++) { + failures.add(f[i]); + } + } + + @Override + public Optional getMasterObserver() { + return Optional.of(this); + } + + @Override + public void preSnapshot(final ObserverContext ctx, + final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor) + throws IOException { + if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) { + throw new IOException("preSnapshot"); + } + } + + @Override + public void preDeleteSnapshot(ObserverContext ctx, + SnapshotDescription snapshot) throws IOException { + if (failures.contains(Failure.PRE_DELETE_SNAPSHOT_FAILURE)) { + throw new IOException("preDeleteSnapshot"); + } + } + + @Override + public void postDeleteSnapshot(ObserverContext ctx, + SnapshotDescription snapshot) throws IOException { + if (failures.contains(Failure.POST_DELETE_SNAPSHOT_FAILURE)) { + throw new IOException("postDeleteSnapshot"); + } + } + } + + /** + * Setup Cluster with appropriate configurations before running tests. + * @throws Exception if starting the mini cluster or setting up the tables fails + */ + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + conf1 = TEST_UTIL.getConfiguration(); + conf1.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MasterSnapshotObserver.class.getName()); + conf1.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + setUpHelper(); + } + + private MasterSnapshotObserver getMasterSnapshotObserver() { + return TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost() + .findCoprocessor(MasterSnapshotObserver.class); + } + + @Test + public void testBackupDeleteWithFailures() throws Exception { + testBackupDeleteWithFailuresAfter(1, Failure.PRE_DELETE_SNAPSHOT_FAILURE); + testBackupDeleteWithFailuresAfter(0, Failure.POST_DELETE_SNAPSHOT_FAILURE); + testBackupDeleteWithFailuresAfter(1, Failure.PRE_SNAPSHOT_FAILURE); + } + + private void testBackupDeleteWithFailuresAfter(int expected, Failure... failures) + throws Exception { + LOG.info("test repair backup delete on a single table with data and failures " + failures[0]); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + String[] backupIds = new String[] { backupId }; + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo info = table.readBackupInfo(backupId); + Path path = new Path(info.getBackupRootDir(), backupId); + FileSystem fs = FileSystem.get(path.toUri(), conf1); + assertTrue(fs.exists(path)); + + Connection conn = TEST_UTIL.getConnection(); + Admin admin = conn.getAdmin(); + MasterSnapshotObserver observer = getMasterSnapshotObserver(); + + observer.setFailures(failures); + try { + getBackupAdmin().deleteBackups(backupIds); + } catch (IOException e) { + if (expected != 1) { + assertTrue(false); + } + } + + // Verify that history length == expected after delete failure + assertTrue(table.getBackupHistory().size() == expected); + + String[] ids = table.getListOfBackupIdsFromDeleteOperation(); + + // Verify that we still have delete record in backup system table + if (expected == 1) { + assertTrue(ids.length == 1); + assertTrue(ids[0].equals(backupId)); + } else { + assertNull(ids); + } + + // Now run repair command to repair "failed" delete operation + String[] args = new String[] { "repair" }; + + observer.setFailures(Failure.NO_FAILURES); + + // Run repair + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + // Verify that history length == 0 + assertTrue(table.getBackupHistory().size() == 0); + ids = table.getListOfBackupIdsFromDeleteOperation(); + + // Verify that we do not have delete record in backup system table + assertNull(ids); + + table.close(); + admin.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java new file mode 100644 index 000000000000..7ce039fd6668 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java @@ -0,0 +1,110 @@ +/* + * 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.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupCommands; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDescribe extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDescribe.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupDescribe.class); + + /** + * Verify that describe works as expected if incorrect backup Id is supplied. + * @throws Exception if creating the {@link BackupDriver} fails + */ + @Test + public void testBackupDescribe() throws Exception { + LOG.info("test backup describe on a single table with data"); + + String[] args = new String[] { "describe", "backup_2" }; + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret < 0); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setErr(new PrintStream(baos)); + args = new String[] { "progress" }; + ToolRunner.run(TEST_UTIL.getConfiguration(), new BackupDriver(), args); + + String output = baos.toString(); + LOG.info("Output from progress: " + output); + assertTrue(output.indexOf(BackupCommands.NO_ACTIVE_SESSION_FOUND) >= 0); + } + + @Test + public void testBackupSetCommandWithNonExistentTable() throws Exception { + String[] args = new String[] { "set", "add", "some_set", "table" }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertNotEquals(ret, 0); + } + + @Test + public void testBackupDescribeCommand() throws Exception { + LOG.info("test backup describe on a single table with data: command-line"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + BackupInfo info = getBackupAdmin().getBackupInfo(backupId); + assertTrue(info.getState() == BackupState.COMPLETE); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "describe", backupId }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + String response = baos.toString(); + assertTrue(response.indexOf(backupId) > 0); + assertTrue(response.indexOf("COMPLETE") > 0); + + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo status = table.readBackupInfo(backupId); + String desc = status.getShortDescription(); + table.close(); + assertTrue(response.indexOf(desc) >= 0); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java new file mode 100644 index 000000000000..1e3c84ec4b06 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java @@ -0,0 +1,148 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.IdentityHashMap; +import java.util.List; +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.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ MasterTests.class, SmallTests.class }) +public class TestBackupHFileCleaner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupHFileCleaner.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupHFileCleaner.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static Configuration conf = TEST_UTIL.getConfiguration(); + private static TableName tableName = TableName.valueOf("backup.hfile.cleaner"); + private static String famName = "fam"; + static FileSystem fs = null; + Path root; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniCluster(1); + fs = FileSystem.get(conf); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + if (fs != null) { + fs.close(); + } + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void setup() throws IOException { + root = TEST_UTIL.getDataTestDirOnTestFS(); + } + + @After + public void cleanup() { + try { + fs.delete(root, true); + } catch (IOException e) { + LOG.warn("Failed to delete files recursively from path " + root); + } + } + + @Test + public void testGetDeletableFiles() throws IOException { + // 1. Create a file + Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs"); + fs.createNewFile(file); + // 2. Assert file is successfully created + assertTrue("Test file not created!", fs.exists(file)); + BackupHFileCleaner cleaner = new BackupHFileCleaner(); + cleaner.setConf(conf); + cleaner.setCheckForFullyBackedUpTables(false); + List stats = new ArrayList<>(); + // Prime the cleaner + cleaner.getDeletableFiles(stats); + // 3. Assert that file as is should be deletable + FileStatus stat = fs.getFileStatus(file); + stats.add(stat); + Iterable deletable = cleaner.getDeletableFiles(stats); + boolean found = false; + for (FileStatus stat1 : deletable) { + if (stat.equals(stat1)) { + found = true; + } + } + assertTrue( + "Cleaner should allow to delete this file as there is no hfile reference " + "for it.", + found); + + // 4. Add the file as bulk load + List list = new ArrayList<>(1); + list.add(file); + try (Connection conn = ConnectionFactory.createConnection(conf); + BackupSystemTable sysTbl = new BackupSystemTable(conn)) { + List sTableList = new ArrayList<>(); + sTableList.add(tableName); + @SuppressWarnings("unchecked") + IdentityHashMap>[] maps = new IdentityHashMap[1]; + maps[0] = new IdentityHashMap<>(); + maps[0].put(Bytes.toBytes(famName), list); + sysTbl.writeBulkLoadedFiles(sTableList, maps, "1"); + } + + // 5. Assert file should not be deletable + deletable = cleaner.getDeletableFiles(stats); + found = false; + for (FileStatus stat1 : deletable) { + if (stat.equals(stat1)) { + found = true; + } + } + assertFalse( + "Cleaner should not allow to delete this file as there is a hfile reference " + "for it.", + found); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupManager.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupManager.java new file mode 100644 index 000000000000..cfbc25cef3ea --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupManager.java @@ -0,0 +1,136 @@ +/* + * 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.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLongArray; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles; + +@Category(MediumTests.class) +public class TestBackupManager { + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupManager.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupManager.class); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + protected static Configuration conf = UTIL.getConfiguration(); + protected static MiniHBaseCluster cluster; + protected static Connection conn; + protected BackupManager backupManager; + + @BeforeClass + public static void setUp() throws Exception { + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf); + BackupManager.decorateRegionServerConfiguration(conf); + cluster = UTIL.startMiniCluster(); + conn = UTIL.getConnection(); + } + + @AfterClass + public static void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Before + public void before() throws IOException { + backupManager = new BackupManager(conn, conn.getConfiguration()); + } + + @After + public void after() { + backupManager.close(); + } + + AtomicLongArray startTimes = new AtomicLongArray(2); + AtomicLongArray stopTimes = new AtomicLongArray(2); + + @Test + public void testStartBackupExclusiveOperation() { + + long sleepTime = 2000; + Runnable r = new Runnable() { + @Override + public void run() { + try { + backupManager.startBackupSession(); + boolean result = startTimes.compareAndSet(0, 0, EnvironmentEdgeManager.currentTime()); + if (!result) { + result = startTimes.compareAndSet(1, 0, EnvironmentEdgeManager.currentTime()); + if (!result) { + throw new IOException("PANIC! Unreachable code"); + } + } + Thread.sleep(sleepTime); + result = stopTimes.compareAndSet(0, 0, EnvironmentEdgeManager.currentTime()); + if (!result) { + result = stopTimes.compareAndSet(1, 0, EnvironmentEdgeManager.currentTime()); + if (!result) { + throw new IOException("PANIC! Unreachable code"); + } + } + backupManager.finishBackupSession(); + } catch (IOException | InterruptedException e) { + fail("Unexpected exception: " + e.getMessage()); + } + } + }; + + Thread[] workers = new Thread[2]; + for (int i = 0; i < workers.length; i++) { + workers[i] = new Thread(r); + workers[i].start(); + } + + for (int i = 0; i < workers.length; i++) { + Uninterruptibles.joinUninterruptibly(workers[i]); + } + LOG.info("Diff start time=" + (startTimes.get(1) - startTimes.get(0)) + "ms"); + LOG.info("Diff finish time=" + (stopTimes.get(1) - stopTimes.get(0)) + "ms"); + assertTrue(startTimes.get(1) - startTimes.get(0) >= sleepTime); + assertTrue(stopTimes.get(1) - stopTimes.get(0) >= sleepTime); + + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java new file mode 100644 index 000000000000..c34f6be43b5e --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMerge.java @@ -0,0 +1,127 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupMerge extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupMerge.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupMerge.class); + + @Test + public void TestIncBackupMergeRestore() throws Exception { + int ADD_ROWS = 99; + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + // Set custom Merge Job implementation + + Connection conn = ConnectionFactory.createConnection(conf1); + + Admin admin = conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table1 + Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); + LOG.debug("writing " + ADD_ROWS + " rows to " + table1); + + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS); + t1.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table1); + + Table t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS); + + Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS); + t2.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table2); + + // #3 - incremental backup for multiple tables + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdIncMultiple)); + + t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS); + t1.close(); + + t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS); + t2.close(); + + // #3 - incremental backup for multiple tables + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple2)); + + try (BackupAdmin bAdmin = new BackupAdminImpl(conn)) { + String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 }; + bAdmin.mergeBackups(backups); + } + + // #6 - restore incremental backup for multiple tables, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2, false, + tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + + Table hTable = conn.getTable(table1_restore); + LOG.debug("After incremental restore: " + hTable.getDescriptor()); + int countRows = TEST_UTIL.countRows(hTable, famName); + LOG.debug("f1 has " + countRows + " rows"); + Assert.assertEquals(NB_ROWS_IN_BATCH + 2 * ADD_ROWS, countRows); + + hTable.close(); + + hTable = conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS); + hTable.close(); + + admin.close(); + conn.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java new file mode 100644 index 000000000000..36cecd3faf58 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java @@ -0,0 +1,162 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +/** + * Create multiple backups for two tables: table1, table2 then perform 1 delete + */ +@Category(LargeTests.class) +public class TestBackupMultipleDeletes extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupMultipleDeletes.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupMultipleDeletes.class); + + @Test + public void testBackupMultipleDeletes() throws Exception { + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + List tables = Lists.newArrayList(table1, table2); + Connection conn = ConnectionFactory.createConnection(conf1); + Admin admin = conn.getAdmin(); + BackupAdmin client = new BackupAdminImpl(conn); + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull)); + // #2 - insert some data to table table1 + Table t1 = conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2); + t1.close(); + // #3 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc1 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc1)); + // #4 - insert some data to table table2 + Table t2 = conn.getTable(table2); + Put p2 = null; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + // #5 - incremental backup for table1, table2 + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc2)); + // #6 - insert some data to table table1 + t1 = conn.getTable(table1); + for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + // #7 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc3 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc3)); + // #8 - insert some data to table table2 + t2 = conn.getTable(table2); + for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) { + p2 = new Put(Bytes.toBytes("row-t1" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + // #9 - incremental backup for table1, table2 + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc4 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc4)); + // #10 full backup for table3 + tables = Lists.newArrayList(table3); + request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull2)); + // #11 - incremental backup for table3 + tables = Lists.newArrayList(table3); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc5 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc5)); + LOG.error("Delete backupIdInc2"); + client.deleteBackups(new String[] { backupIdInc2 }); + LOG.error("Delete backupIdInc2 done"); + List list = client.getHistory(100); + // First check number of backup images before and after + assertEquals(4, list.size()); + // then verify that no backupIdInc2,3,4 + Set ids = new HashSet(); + ids.add(backupIdInc2); + ids.add(backupIdInc3); + ids.add(backupIdInc4); + for (BackupInfo info : list) { + String backupId = info.getBackupId(); + if (ids.contains(backupId)) { + assertTrue(false); + } + } + // Verify that backupInc5 contains only table3 + boolean found = false; + for (BackupInfo info : list) { + String backupId = info.getBackupId(); + if (backupId.equals(backupIdInc5)) { + assertTrue(info.getTables().size() == 1); + assertEquals(table3, info.getTableNames().get(0)); + found = true; + } + } + assertTrue(found); + admin.close(); + conn.close(); + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java new file mode 100644 index 000000000000..239e1409bdd8 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java @@ -0,0 +1,92 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestBackupRepair extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupRepair.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupRepair.class); + + @Test + public void testFullBackupWithFailuresAndRestore() throws Exception { + + autoRestoreOnFailure = false; + + conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS, + FullTableBackupClientForTest.class.getName()); + int maxStage = Stage.values().length - 1; + // Fail stage in loop between 0 and 4 inclusive + for (int stage = 0; stage < maxStage; stage++) { + LOG.info("Running stage " + stage); + runBackupAndFailAtStageWithRestore(stage); + } + } + + public void runBackupAndFailAtStageWithRestore(int stage) throws Exception { + + conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage); + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() + "," + table2.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertFalse(ret == 0); + + // Now run restore + args = new String[] { "repair" }; + + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + + List backups = table.getBackupHistory(); + int after = table.getBackupHistory().size(); + + assertTrue(after == before + 1); + for (BackupInfo data : backups) { + String backupId = data.getBackupId(); + assertFalse(checkSucceeded(backupId)); + } + Set tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR); + assertTrue(tables.size() == 0); + } + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java new file mode 100644 index 000000000000..fa624250929d --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java @@ -0,0 +1,141 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupShowHistory extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupShowHistory.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupShowHistory.class); + + private boolean findBackup(List history, String backupId) { + assertTrue(history.size() > 0); + boolean success = false; + for (BackupInfo info : history) { + if (info.getBackupId().equals(backupId)) { + success = true; + break; + } + } + return success; + } + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testBackupHistory() throws Exception { + + LOG.info("test backup history on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + + List history = getBackupAdmin().getHistory(10); + assertTrue(findBackup(history, backupId)); + BackupInfo.Filter nullFilter = info -> true; + history = BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), nullFilter); + assertTrue(findBackup(history, backupId)); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + LOG.info("show_history"); + String output = baos.toString(); + LOG.info(output); + baos.close(); + assertTrue(output.indexOf(backupId) > 0); + + tableList = Lists.newArrayList(table2); + String backupId2 = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId2)); + LOG.info("backup complete: " + table2); + BackupInfo.Filter tableNameFilter = image -> { + if (table1 == null) { + return true; + } + + List names = image.getTableNames(); + return names.contains(table1); + }; + BackupInfo.Filter tableSetFilter = info -> { + String backupId1 = info.getBackupId(); + return backupId1.startsWith("backup"); + }; + + history = getBackupAdmin().getHistory(10, tableNameFilter, tableSetFilter); + assertTrue(history.size() > 0); + boolean success = true; + for (BackupInfo info : history) { + if (!info.getTableNames().contains(table1)) { + success = false; + break; + } + } + assertTrue(success); + + history = + BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), tableNameFilter, tableSetFilter); + assertTrue(history.size() > 0); + success = true; + for (BackupInfo info : history) { + if (!info.getTableNames().contains(table1)) { + success = false; + break; + } + } + assertTrue(success); + + args = + new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR, "-t", "table1", "-s", "backup" }; + // Run backup + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + LOG.info("show_history"); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSmallTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSmallTests.java new file mode 100644 index 000000000000..83cc19578ade --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSmallTests.java @@ -0,0 +1,60 @@ +/* + * 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.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBackupSmallTests extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupSmallTests.class); + + private static final UserGroupInformation DIANA = + UserGroupInformation.createUserForTesting("diana", new String[] {}); + private static final String PERMISSION_TEST_PATH = Path.SEPARATOR + "permissionUT"; + + @Test + public void testBackupPathIsAccessible() throws Exception { + Path path = new Path(PERMISSION_TEST_PATH); + FileSystem fs = FileSystem.get(TEST_UTIL.getConnection().getConfiguration()); + fs.mkdirs(path); + } + + @Test(expected = IOException.class) + public void testBackupPathIsNotAccessible() throws Exception { + Path path = new Path(PERMISSION_TEST_PATH); + FileSystem rootFs = FileSystem.get(TEST_UTIL.getConnection().getConfiguration()); + rootFs.mkdirs(path.getParent()); + rootFs.setPermission(path.getParent(), FsPermission.createImmutable((short) 000)); + FileSystem fs = + DFSTestUtil.getFileSystemAs(DIANA, TEST_UTIL.getConnection().getConfiguration()); + fs.mkdirs(path); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java new file mode 100644 index 000000000000..1a1e5dbf1cc1 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java @@ -0,0 +1,97 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupStatusProgress extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupStatusProgress.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupStatusProgress.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testBackupStatusProgress() throws Exception { + LOG.info("test backup status/progress on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + BackupInfo info = getBackupAdmin().getBackupInfo(backupId); + assertTrue(info.getState() == BackupState.COMPLETE); + + LOG.debug(info.getShortDescription()); + assertTrue(info.getProgress() > 0); + + } + + @Test + public void testBackupStatusProgressCommand() throws Exception { + LOG.info("test backup status/progress on a single table with data: command-line"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "describe", backupId }; + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + String responce = baos.toString(); + assertTrue(responce.indexOf(backupId) > 0); + assertTrue(responce.indexOf("COMPLETE") > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + args = new String[] { "progress", backupId }; + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + responce = baos.toString(); + assertTrue(responce.indexOf(backupId) >= 0); + assertTrue(responce.indexOf("progress") > 0); + assertTrue(responce.indexOf("100") > 0); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java new file mode 100644 index 000000000000..301ae2a5985e --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java @@ -0,0 +1,504 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test cases for backup system table API + */ +@Category(MediumTests.class) +public class TestBackupSystemTable { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupSystemTable.class); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + protected static Configuration conf = UTIL.getConfiguration(); + protected static MiniHBaseCluster cluster; + protected static Connection conn; + protected BackupSystemTable table; + + @BeforeClass + public static void setUp() throws Exception { + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf); + BackupManager.decorateRegionServerConfiguration(conf); + cluster = UTIL.startMiniCluster(); + conn = UTIL.getConnection(); + } + + @Before + public void before() throws IOException { + table = new BackupSystemTable(conn); + } + + @After + public void after() { + if (table != null) { + table.close(); + } + + } + + @Test + public void testUpdateReadDeleteBackupStatus() throws IOException { + BackupInfo ctx = createBackupInfo(); + table.updateBackupInfo(ctx); + BackupInfo readCtx = table.readBackupInfo(ctx.getBackupId()); + assertTrue(compare(ctx, readCtx)); + // try fake backup id + readCtx = table.readBackupInfo("fake"); + assertNull(readCtx); + // delete backup info + table.deleteBackupInfo(ctx.getBackupId()); + readCtx = table.readBackupInfo(ctx.getBackupId()); + assertNull(readCtx); + cleanBackupTable(); + } + + @Test + public void testWriteReadBackupStartCode() throws IOException { + long code = 100L; + table.writeBackupStartCode(code, "root"); + String readCode = table.readBackupStartCode("root"); + assertEquals(code, Long.parseLong(readCode)); + cleanBackupTable(); + } + + private void cleanBackupTable() throws IOException { + Admin admin = UTIL.getAdmin(); + admin.disableTable(BackupSystemTable.getTableName(conf)); + admin.truncateTable(BackupSystemTable.getTableName(conf), true); + if (admin.isTableDisabled(BackupSystemTable.getTableName(conf))) { + admin.enableTable(BackupSystemTable.getTableName(conf)); + } + } + + @Test + public void testBackupHistory() throws Exception { + int n = 10; + List list = createBackupInfoList(n); + + // Load data + for (BackupInfo bc : list) { + // Make sure we set right status + bc.setState(BackupState.COMPLETE); + table.updateBackupInfo(bc); + } + + // Reverse list for comparison + Collections.reverse(list); + List history = table.getBackupHistory(); + assertTrue(history.size() == n); + + for (int i = 0; i < n; i++) { + BackupInfo ctx = list.get(i); + BackupInfo data = history.get(i); + assertTrue(compare(ctx, data)); + } + + cleanBackupTable(); + + } + + @Test + public void testBackupDelete() throws Exception { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + int n = 10; + List list = createBackupInfoList(n); + + // Load data + for (BackupInfo bc : list) { + // Make sure we set right status + bc.setState(BackupState.COMPLETE); + table.updateBackupInfo(bc); + } + + // Verify exists + for (BackupInfo bc : list) { + assertNotNull(table.readBackupInfo(bc.getBackupId())); + } + + // Delete all + for (BackupInfo bc : list) { + table.deleteBackupInfo(bc.getBackupId()); + } + + // Verify do not exists + for (BackupInfo bc : list) { + assertNull(table.readBackupInfo(bc.getBackupId())); + } + + cleanBackupTable(); + } + + } + + @Test + public void testRegionServerLastLogRollResults() throws IOException { + String[] servers = new String[] { "server1", "server2", "server3" }; + Long[] timestamps = new Long[] { 100L, 102L, 107L }; + + for (int i = 0; i < servers.length; i++) { + table.writeRegionServerLastLogRollResult(servers[i], timestamps[i], "root"); + } + + HashMap result = table.readRegionServerLastLogRollResult("root"); + assertTrue(servers.length == result.size()); + Set keys = result.keySet(); + String[] keysAsArray = new String[keys.size()]; + keys.toArray(keysAsArray); + Arrays.sort(keysAsArray); + + for (int i = 0; i < keysAsArray.length; i++) { + assertEquals(keysAsArray[i], servers[i]); + Long ts1 = timestamps[i]; + Long ts2 = result.get(keysAsArray[i]); + assertEquals(ts1, ts2); + } + + cleanBackupTable(); + } + + @Test + public void testIncrementalBackupTableSet() throws IOException { + TreeSet tables1 = new TreeSet<>(); + + tables1.add(TableName.valueOf("t1")); + tables1.add(TableName.valueOf("t2")); + tables1.add(TableName.valueOf("t3")); + + TreeSet tables2 = new TreeSet<>(); + + tables2.add(TableName.valueOf("t3")); + tables2.add(TableName.valueOf("t4")); + tables2.add(TableName.valueOf("t5")); + + table.addIncrementalBackupTableSet(tables1, "root"); + + try (BackupSystemTable systemTable = new BackupSystemTable(conn)) { + TreeSet res1 = + (TreeSet) systemTable.getIncrementalBackupTableSet("root"); + assertTrue(tables1.size() == res1.size()); + Iterator desc1 = tables1.descendingIterator(); + Iterator desc2 = res1.descendingIterator(); + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + systemTable.addIncrementalBackupTableSet(tables2, "root"); + TreeSet res2 = + (TreeSet) systemTable.getIncrementalBackupTableSet("root"); + assertTrue((tables2.size() + tables1.size() - 1) == res2.size()); + tables1.addAll(tables2); + desc1 = tables1.descendingIterator(); + desc2 = res2.descendingIterator(); + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + } + + cleanBackupTable(); + } + + @Test + public void testRegionServerLogTimestampMap() throws IOException { + TreeSet tables = new TreeSet<>(); + + tables.add(TableName.valueOf("t1")); + tables.add(TableName.valueOf("t2")); + tables.add(TableName.valueOf("t3")); + + HashMap rsTimestampMap = new HashMap<>(); + + rsTimestampMap.put("rs1:100", 100L); + rsTimestampMap.put("rs2:100", 101L); + rsTimestampMap.put("rs3:100", 103L); + + table.writeRegionServerLogTimestamp(tables, rsTimestampMap, "root"); + + Map> result = table.readLogTimestampMap("root"); + + assertTrue(tables.size() == result.size()); + + for (TableName t : tables) { + Map rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1:100"), Long.valueOf(100L)); + assertEquals(rstm.get("rs2:100"), Long.valueOf(101L)); + assertEquals(rstm.get("rs3:100"), Long.valueOf(103L)); + } + + Set tables1 = new TreeSet<>(); + + tables1.add(TableName.valueOf("t3")); + tables1.add(TableName.valueOf("t4")); + tables1.add(TableName.valueOf("t5")); + + HashMap rsTimestampMap1 = new HashMap<>(); + + rsTimestampMap1.put("rs1:100", 200L); + rsTimestampMap1.put("rs2:100", 201L); + rsTimestampMap1.put("rs3:100", 203L); + + table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1, "root"); + + result = table.readLogTimestampMap("root"); + + assertTrue(5 == result.size()); + + for (TableName t : tables) { + Map rstm = result.get(t); + assertNotNull(rstm); + if (t.equals(TableName.valueOf("t3")) == false) { + assertEquals(rstm.get("rs1:100"), Long.valueOf(100L)); + assertEquals(rstm.get("rs2:100"), Long.valueOf(101L)); + assertEquals(rstm.get("rs3:100"), Long.valueOf(103L)); + } else { + assertEquals(rstm.get("rs1:100"), Long.valueOf(200L)); + assertEquals(rstm.get("rs2:100"), Long.valueOf(201L)); + assertEquals(rstm.get("rs3:100"), Long.valueOf(203L)); + } + } + + for (TableName t : tables1) { + Map rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1:100"), Long.valueOf(200L)); + assertEquals(rstm.get("rs2:100"), Long.valueOf(201L)); + assertEquals(rstm.get("rs3:100"), Long.valueOf(203L)); + } + + cleanBackupTable(); + + } + + /** + * Backup set tests + */ + + @Test + public void testBackupSetAddNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals(tables[i])); + } + cleanBackupTable(); + } + + } + + @Test + public void testBackupSetAddExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] addTables = new String[] { "table4", "table5", "table6" }; + table.addToBackupSet(setName, addTables); + + Set expectedTables = + new HashSet<>(Arrays.asList("table1", "table2", "table3", "table4", "table5", "table6")); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == expectedTables.size()); + for (TableName tableName : tnames) { + assertTrue(expectedTables.remove(tableName.getNameAsString())); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetAddExistsIntersects() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] addTables = new String[] { "table3", "table4", "table5", "table6" }; + table.addToBackupSet(setName, addTables); + + Set expectedTables = + new HashSet<>(Arrays.asList("table1", "table2", "table3", "table4", "table5", "table6")); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == expectedTables.size()); + for (TableName tableName : tnames) { + assertTrue(expectedTables.remove(tableName.getNameAsString())); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemoveSomeNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] removeTables = new String[] { "table4", "table5", "table6" }; + table.removeFromBackupSet(setName, removeTables); + + Set expectedTables = new HashSet<>(Arrays.asList("table1", "table2", "table3")); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == expectedTables.size()); + for (TableName tableName : tnames) { + assertTrue(expectedTables.remove(tableName.getNameAsString())); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemove() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] removeTables = new String[] { "table4", "table3" }; + table.removeFromBackupSet(setName, removeTables); + + Set expectedTables = new HashSet<>(Arrays.asList("table1", "table2")); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == expectedTables.size()); + for (TableName tableName : tnames) { + assertTrue(expectedTables.remove(tableName.getNameAsString())); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetDelete() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + table.deleteBackupSet(setName); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames == null); + cleanBackupTable(); + } + } + + @Test + public void testBackupSetList() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName1 = "name1"; + String setName2 = "name2"; + table.addToBackupSet(setName1, tables); + table.addToBackupSet(setName2, tables); + + List list = table.listBackupSets(); + + assertTrue(list.size() == 2); + assertTrue(list.get(0).equals(setName1)); + assertTrue(list.get(1).equals(setName2)); + + cleanBackupTable(); + } + } + + private boolean compare(BackupInfo one, BackupInfo two) { + return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType()) + && one.getBackupRootDir().equals(two.getBackupRootDir()) + && one.getStartTs() == two.getStartTs() && one.getCompleteTs() == two.getCompleteTs(); + } + + private BackupInfo createBackupInfo() { + BackupInfo ctxt = new BackupInfo("backup_" + System.nanoTime(), BackupType.FULL, + new TableName[] { TableName.valueOf("t1"), TableName.valueOf("t2"), TableName.valueOf("t3") }, + "/hbase/backup"); + ctxt.setStartTs(EnvironmentEdgeManager.currentTime()); + ctxt.setCompleteTs(EnvironmentEdgeManager.currentTime() + 1); + return ctxt; + } + + private List createBackupInfoList(int size) throws InterruptedException { + List list = new ArrayList<>(); + for (int i = 0; i < size; i++) { + list.add(createBackupInfo()); + // XXX Why do we need this sleep? + Thread.sleep(10); + } + return list; + } + + @AfterClass + public static void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupUtils.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupUtils.java new file mode 100644 index 000000000000..023acd79fb89 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupUtils.java @@ -0,0 +1,81 @@ +/* + * 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.security.PrivilegedAction; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(SmallTests.class) +public class TestBackupUtils { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(TestBackupUtils.class); + + protected static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected static Configuration conf = TEST_UTIL.getConfiguration(); + + @Test + public void TestGetBulkOutputDir() { + // Create a user who is not the current user + String fooUserName = "foo1234"; + String fooGroupName = "group1"; + UserGroupInformation ugi = + UserGroupInformation.createUserForTesting(fooUserName, new String[] { fooGroupName }); + // Get user's home directory + Path fooHomeDirectory = ugi.doAs(new PrivilegedAction() { + @Override + public Path run() { + try (FileSystem fs = FileSystem.get(conf)) { + return fs.getHomeDirectory(); + } catch (IOException ioe) { + LOG.error("Failed to get foo's home directory", ioe); + } + return null; + } + }); + + Path bulkOutputDir = ugi.doAs(new PrivilegedAction() { + @Override + public Path run() { + try { + return BackupUtils.getBulkOutputDir("test", conf, false); + } catch (IOException ioe) { + LOG.error("Failed to get bulk output dir path", ioe); + } + return null; + } + }); + // Make sure the directory is in foo1234's home directory + Assert.assertTrue(bulkOutputDir.toString().startsWith(fooHomeDirectory.toString())); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java new file mode 100644 index 000000000000..7cec06799742 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java @@ -0,0 +1,63 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestFullBackup extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullBackup.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFullBackup.class); + + @Test + public void testFullBackupMultipleCommand() throws Exception { + LOG.info("test full backup on a multiple tables with data: command-line"); + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() + "," + table2.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + int after = table.getBackupHistory().size(); + assertTrue(after == before + 1); + for (BackupInfo data : backups) { + String backupId = data.getBackupId(); + assertTrue(checkSucceeded(backupId)); + } + } + LOG.info("backup complete"); + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java new file mode 100644 index 000000000000..af6ce077e051 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java @@ -0,0 +1,100 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestFullBackupSet extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullBackupSet.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFullBackupSet.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullBackupSetExist() throws Exception { + LOG.info("Test full backup, backup set exists"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + assertTrue(backups.size() == 1); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + // Restore from set into other table + args = new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m", + table1_restore.getNameAsString(), "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + // Verify number of rows in both tables + assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + LOG.info("restore into other table is complete"); + hba.close(); + } + } + + @Test + public void testFullBackupSetDoesNotExist() throws Exception { + LOG.info("test full backup, backup set does not exist"); + String name = "name1"; + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret != 0); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java new file mode 100644 index 000000000000..98e05cc5a128 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java @@ -0,0 +1,131 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestFullBackupSetRestoreSet extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullBackupSetRestoreSet.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFullBackupSetRestoreSet.class); + + @Test + public void testFullRestoreSetToOtherTable() throws Exception { + + LOG.info("Test full restore set"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + assertTrue(backups.size() == 1); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + // Restore from set into other table + args = new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m", + table1_restore.getNameAsString(), "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + // Verify number of rows in both tables + assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + LOG.info("restore into other table is complete"); + hba.close(); + } + } + + @Test + public void testFullRestoreSetToSameTable() throws Exception { + + LOG.info("Test full restore set to same table"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name1"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + int count = TEST_UTIL.countRows(table1); + TEST_UTIL.deleteTable(table1); + + // Restore from set into other table + args = new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1)); + // Verify number of rows in both tables + assertEquals(count, TEST_UTIL.countRows(table1)); + LOG.info("restore into same table is complete"); + hba.close(); + + } + + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java new file mode 100644 index 000000000000..1536fd1841fb --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java @@ -0,0 +1,82 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestFullBackupWithFailures extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullBackupWithFailures.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFullBackupWithFailures.class); + + @Test + public void testFullBackupWithFailures() throws Exception { + conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS, + FullTableBackupClientForTest.class.getName()); + int maxStage = Stage.values().length - 1; + // Fail stages between 0 and 4 inclusive + for (int stage = 0; stage <= maxStage; stage++) { + LOG.info("Running stage " + stage); + runBackupAndFailAtStage(stage); + } + } + + public void runBackupAndFailAtStage(int stage) throws Exception { + + conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage); + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() + "," + table2.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertFalse(ret == 0); + List backups = table.getBackupHistory(); + int after = table.getBackupHistory().size(); + + assertTrue(after == before + 1); + for (BackupInfo data : backups) { + String backupId = data.getBackupId(); + assertFalse(checkSucceeded(backupId)); + } + Set tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR); + assertTrue(tables.size() == 0); + } + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java new file mode 100644 index 000000000000..385a6b3c5193 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java @@ -0,0 +1,337 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestFullRestore extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFullRestore.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestFullRestore.class); + + /** + * Verify that a single table is restored to a new table. + * @throws Exception if doing the backup, restoring it or an operation on the tables fails + */ + @Test + public void testFullRestoreSingle() throws Exception { + LOG.info("test full restore on a single table empty table"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, + tablemap, false)); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + hba.close(); + } + + @Test + public void testFullRestoreSingleCommand() throws Exception { + LOG.info("test full restore on a single table empty table: command-line"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + // restore [tableMapping] + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m", + table1_restore.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + hba.close(); + } + + @Test + public void testFullRestoreCheckCommand() throws Exception { + LOG.info("test full restore on a single table: command-line, check only"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + // restore [tableMapping] + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m", + table1_restore.getNameAsString(), "-c" }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + // Verify that table has not been restored + Admin hba = TEST_UTIL.getAdmin(); + assertFalse(hba.tableExists(table1_restore)); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception if doing the backup, restoring it or an operation on the tables fails + */ + @Test + public void testFullRestoreMultiple() throws Exception { + LOG.info("create full backup image on multiple tables"); + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, tablemap, false)); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception if doing the backup, restoring it or an operation on the tables fails + */ + @Test + public void testFullRestoreMultipleCommand() throws Exception { + LOG.info("create full backup image on multiple tables: command-line"); + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + + // restore [tableMapping] + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, "-t", + StringUtils.join(restore_tableset, ","), "-m", StringUtils.join(tablemap, ",") }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } + + /** + * Verify that a single table is restored using overwrite. + * @throws Exception if doing the backup or restoring it fails + */ + @Test + public void testFullRestoreSingleOverwrite() throws Exception { + LOG.info("test full restore on a single table empty table"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { table1 }; + BackupAdmin client = getBackupAdmin(); + client.restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, null, true)); + } + + /** + * Verify that a single table is restored using overwrite. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullRestoreSingleOverwriteCommand() throws Exception { + LOG.info("test full restore on a single table empty table: command-line"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", StringUtils.join(tableset, ","), "-o" }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1)); + hba.close(); + } + + /** + * Verify that multiple tables are restored to new tables using overwrite. + * @throws Exception if doing the backup or restoring it fails + */ + @Test + public void testFullRestoreMultipleOverwrite() throws Exception { + LOG.info("create full backup image on multiple tables"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, null, true)); + } + + /** + * Verify that multiple tables are restored to new tables using overwrite. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullRestoreMultipleOverwriteCommand() throws Exception { + LOG.info("create full backup image on multiple tables: command-line"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + // restore [tableMapping] + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, "-t", + StringUtils.join(restore_tableset, ","), "-o" }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table2)); + assertTrue(hba.tableExists(table3)); + hba.close(); + } + + /** + * Verify that restore fails on a single table that does not exist. + * @throws Exception if doing the backup or restoring it fails + */ + @Test(expected = IOException.class) + public void testFullRestoreSingleDNE() throws Exception { + LOG.info("test restore fails on a single table that does not exist"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { TableName.valueOf("faketable") }; + TableName[] tablemap = new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, + tablemap, false)); + } + + /** + * Verify that restore fails on a single table that does not exist. + * @throws Exception if doing the backup or restoring it fails + */ + @Test + public void testFullRestoreSingleDNECommand() throws Exception { + LOG.info("test restore fails on a single table that does not exist: command-line"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { TableName.valueOf("faketable") }; + TableName[] tablemap = new TableName[] { table1_restore }; + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, StringUtils.join(tableset, ","), "-m", + StringUtils.join(tablemap, ",") }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret != 0); + } + + /** + * Verify that restore fails on multiple tables that do not exist. + * @throws Exception if doing the backup or restoring it fails + */ + @Test(expected = IOException.class) + public void testFullRestoreMultipleDNE() throws Exception { + LOG.info("test restore fails on multiple tables that do not exist"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = + new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, tablemap, false)); + } + + /** + * Verify that restore fails on multiple tables that do not exist. + * @throws Exception if doing the backup or restoring it fails + */ + @Test + public void testFullRestoreMultipleDNECommand() throws Exception { + LOG.info("test restore fails on multiple tables that do not exist: command-line"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = + new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + String[] args = new String[] { BACKUP_ROOT_DIR, backupId, + StringUtils.join(restore_tableset, ","), "-m", StringUtils.join(tablemap, ",") }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret != 0); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java new file mode 100644 index 000000000000..619e6b14e644 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -0,0 +1,219 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +@RunWith(Parameterized.class) +public class TestIncrementalBackup extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementalBackup.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestIncrementalBackup.class); + + @Parameterized.Parameters + public static Collection data() { + provider = "multiwal"; + List params = new ArrayList<>(); + params.add(new Object[] { Boolean.TRUE }); + return params; + } + + public TestIncrementalBackup(Boolean b) { + } + + // implement all test cases in 1 test since incremental + // backup/restore has dependencies + @Test + public void TestIncBackupRestore() throws Exception { + int ADD_ROWS = 99; + + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + List tables = Lists.newArrayList(table1, table2); + final byte[] fam3Name = Bytes.toBytes("f3"); + final byte[] mobName = Bytes.toBytes("mob"); + + TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name)) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(mobName).setMobEnabled(true) + .setMobThreshold(5L).build()) + .build(); + TEST_UTIL.getAdmin().modifyTable(newTable1Desc); + + try (Connection conn = ConnectionFactory.createConnection(conf1)) { + int NB_ROWS_FAM3 = 6; + insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close(); + insertIntoTable(conn, table1, mobName, 3, NB_ROWS_FAM3).close(); + Admin admin = conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table + Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); + LOG.debug("writing " + ADD_ROWS + " rows to " + table1); + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3); + LOG.debug("written " + ADD_ROWS + " rows to " + table1); + // additionally, insert rows to MOB cf + int NB_ROWS_MOB = 111; + insertIntoTable(conn, table1, mobName, 3, NB_ROWS_MOB); + LOG.debug("written " + NB_ROWS_MOB + " rows to " + table1 + " to Mob enabled CF"); + t1.close(); + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_MOB); + Table t2 = conn.getTable(table2); + Put p2; + for (int i = 0; i < 5; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + Assert.assertEquals(NB_ROWS_IN_BATCH + 5, TEST_UTIL.countRows(t2)); + t2.close(); + LOG.debug("written " + 5 + " rows to " + table2); + // split table1 + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + List regions = cluster.getRegions(table1); + byte[] name = regions.get(0).getRegionInfo().getRegionName(); + long startSplitTime = EnvironmentEdgeManager.currentTime(); + try { + admin.splitRegionAsync(name).get(); + } catch (Exception e) { + // although split fail, this may not affect following check in current API, + // exception will be thrown. + LOG.debug("region is not splittable, because " + e); + } + while (!admin.isTableAvailable(table1)) { + Thread.sleep(100); + } + long endSplitTime = EnvironmentEdgeManager.currentTime(); + // split finished + LOG.debug("split finished in =" + (endSplitTime - startSplitTime)); + + // #3 - incremental backup for multiple tables + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple)); + + // add column family f2 to table1 + // drop column family f3 + final byte[] fam2Name = Bytes.toBytes("f2"); + newTable1Desc = TableDescriptorBuilder.newBuilder(newTable1Desc) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam2Name)).removeColumnFamily(fam3Name) + .build(); + TEST_UTIL.getAdmin().modifyTable(newTable1Desc); + + int NB_ROWS_FAM2 = 7; + Table t3 = insertIntoTable(conn, table1, fam2Name, 2, NB_ROWS_FAM2); + t3.close(); + + // Wait for 5 sec to make sure that old WALs were deleted + Thread.sleep(5000); + + // #4 - additional incremental backup for multiple tables + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple2)); + + // #5 - restore full backup for all tables + TableName[] tablesRestoreFull = new TableName[] { table1, table2 }; + TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore }; + + LOG.debug("Restoring full " + backupIdFull); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, + tablesRestoreFull, tablesMapFull, true)); + + // #6.1 - check tables for full restore + Admin hAdmin = TEST_UTIL.getAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + assertTrue(hAdmin.tableExists(table2_restore)); + hAdmin.close(); + + // #6.2 - checking row count of tables for full restore + Table hTable = conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + NB_ROWS_FAM3); + hTable.close(); + + hTable = conn.getTable(table2_restore); + Assert.assertEquals(NB_ROWS_IN_BATCH, TEST_UTIL.countRows(hTable)); + hTable.close(); + + // #7 - restore incremental backup for multiple tables, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2, false, + tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + hTable = conn.getTable(table1_restore); + + LOG.debug("After incremental restore: " + hTable.getDescriptor()); + int countFamName = TEST_UTIL.countRows(hTable, famName); + LOG.debug("f1 has " + countFamName + " rows"); + Assert.assertEquals(countFamName, NB_ROWS_IN_BATCH + ADD_ROWS); + + int countFam2Name = TEST_UTIL.countRows(hTable, fam2Name); + LOG.debug("f2 has " + countFam2Name + " rows"); + Assert.assertEquals(countFam2Name, NB_ROWS_FAM2); + + int countMobName = TEST_UTIL.countRows(hTable, mobName); + LOG.debug("mob has " + countMobName + " rows"); + Assert.assertEquals(countMobName, NB_ROWS_MOB); + hTable.close(); + + hTable = conn.getTable(table2_restore); + Assert.assertEquals(NB_ROWS_IN_BATCH + 5, TEST_UTIL.countRows(hTable)); + hTable.close(); + admin.close(); + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java new file mode 100644 index 000000000000..a5eec87fb06b --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java @@ -0,0 +1,129 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +/** + * 1. Create table t1, t2 2. Load data to t1, t2 3 Full backup t1, t2 4 Delete t2 5 Load data to t1 + * 6 Incremental backup t1 + */ +@Category(LargeTests.class) +public class TestIncrementalBackupDeleteTable extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementalBackupDeleteTable.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestIncrementalBackupDeleteTable.class); + + // implement all test cases in 1 test since incremental backup/restore has dependencies + @Test + public void testIncBackupDeleteTable() throws Exception { + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + Connection conn = ConnectionFactory.createConnection(conf1); + Admin admin = conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table table1 + Table t1 = conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2); + t1.close(); + + // Delete table table2 + admin.disableTable(table2); + admin.deleteTable(table2); + + // #3 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple)); + + // #4 - restore full backup for all tables, without overwrite + TableName[] tablesRestoreFull = new TableName[] { table1, table2 }; + + TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore }; + + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, + tablesRestoreFull, tablesMapFull, false)); + + // #5.1 - check tables for full restore + Admin hAdmin = TEST_UTIL.getAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + assertTrue(hAdmin.tableExists(table2_restore)); + + // #5.2 - checking row count of tables for full restore + Table hTable = conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH); + hTable.close(); + + hTable = conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH); + hTable.close(); + + // #6 - restore incremental backup for table1 + TableName[] tablesRestoreIncMultiple = new TableName[] { table1 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false, + tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + + hTable = conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2); + hTable.close(); + admin.close(); + conn.close(); + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java new file mode 100644 index 000000000000..1ece1770489b --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java @@ -0,0 +1,350 @@ +/* + * 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.util.BackupUtils.succeeded; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupCommands; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestIncrementalBackupMergeWithFailures extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementalBackupMergeWithFailures.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestIncrementalBackupMergeWithFailures.class); + + enum FailurePhase { + PHASE1, + PHASE2, + PHASE3, + PHASE4 + } + + public final static String FAILURE_PHASE_KEY = "failurePhase"; + + static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob { + FailurePhase failurePhase; + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + String val = conf.get(FAILURE_PHASE_KEY); + if (val != null) { + failurePhase = FailurePhase.valueOf(val); + } else { + Assert.fail("Failure phase is not set"); + } + } + + /** + * This is the exact copy of parent's run() with injections of different types of failures + */ + @Override + public void run(String[] backupIds) throws IOException { + String bulkOutputConfKey; + + // TODO : run player on remote cluster + player = new MapReduceHFileSplitterJob(); + bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY; + // Player reads all files in arbitrary directory structure and creates + // a Map task for each file + String bids = StringUtils.join(backupIds, ","); + + if (LOG.isDebugEnabled()) { + LOG.debug("Merge backup images " + bids); + } + + List> processedTableList = new ArrayList<>(); + boolean finishedTables = false; + Connection conn = ConnectionFactory.createConnection(getConf()); + BackupSystemTable table = new BackupSystemTable(conn); + FileSystem fs = FileSystem.get(getConf()); + + try { + // Start backup exclusive operation + table.startBackupExclusiveOperation(); + // Start merge operation + table.startMergeOperation(backupIds); + + // Select most recent backup id + String mergedBackupId = BackupUtils.findMostRecentBackupId(backupIds); + + TableName[] tableNames = getTableNamesInBackupImages(backupIds); + + BackupInfo bInfo = table.readBackupInfo(backupIds[0]); + String backupRoot = bInfo.getBackupRootDir(); + // PHASE 1 + checkFailure(FailurePhase.PHASE1); + + for (int i = 0; i < tableNames.length; i++) { + LOG.info("Merge backup images for " + tableNames[i]); + + // Find input directories for table + Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds); + String dirs = StringUtils.join(dirPaths, ","); + Path bulkOutputPath = BackupUtils.getBulkOutputDir( + BackupUtils.getFileNameCompatibleString(tableNames[i]), getConf(), false); + // Delete content if exists + if (fs.exists(bulkOutputPath)) { + if (!fs.delete(bulkOutputPath, true)) { + LOG.warn("Can not delete: " + bulkOutputPath); + } + } + Configuration conf = getConf(); + conf.set(bulkOutputConfKey, bulkOutputPath.toString()); + String[] playerArgs = { dirs, tableNames[i].getNameAsString() }; + + // PHASE 2 + checkFailure(FailurePhase.PHASE2); + player.setConf(getConf()); + int result = player.run(playerArgs); + if (succeeded(result)) { + // Add to processed table list + processedTableList.add(new Pair<>(tableNames[i], bulkOutputPath)); + } else { + throw new IOException("Can not merge backup images for " + dirs + + " (check Hadoop/MR and HBase logs). Player return code =" + result); + } + LOG.debug("Merge Job finished:" + result); + } + List tableList = toTableNameList(processedTableList); + // PHASE 3 + checkFailure(FailurePhase.PHASE3); + table.updateProcessedTablesForMerge(tableList); + finishedTables = true; + + // (modification of a backup file system) + // Move existing mergedBackupId data into tmp directory + // we will need it later in case of a failure + Path tmpBackupDir = + HBackupFileSystem.getBackupTmpDirPathForBackupId(backupRoot, mergedBackupId); + Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, mergedBackupId); + if (!fs.rename(backupDirPath, tmpBackupDir)) { + throw new IOException("Failed to rename " + backupDirPath + " to " + tmpBackupDir); + } else { + LOG.debug("Renamed " + backupDirPath + " to " + tmpBackupDir); + } + // Move new data into backup dest + for (Pair tn : processedTableList) { + moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId); + } + checkFailure(FailurePhase.PHASE4); + // Update backup manifest + List backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId); + updateBackupManifest(tmpBackupDir.getParent().toString(), mergedBackupId, backupsToDelete); + // Copy meta files back from tmp to backup dir + copyMetaData(fs, tmpBackupDir, backupDirPath); + // Delete tmp dir (Rename back during repair) + if (!fs.delete(tmpBackupDir, true)) { + // WARN and ignore + LOG.warn("Could not delete tmp dir: " + tmpBackupDir); + } + // Delete old data + deleteBackupImages(backupsToDelete, conn, fs, backupRoot); + // Finish merge session + table.finishMergeOperation(); + // Release lock + table.finishBackupExclusiveOperation(); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + LOG.error(e.toString(), e); + if (!finishedTables) { + // cleanup bulk directories and finish merge + // merge MUST be repeated (no need for repair) + cleanupBulkLoadDirs(fs, toPathList(processedTableList)); + table.finishMergeOperation(); + table.finishBackupExclusiveOperation(); + throw new IOException("Backup merge operation failed, you should try it again", e); + } else { + // backup repair must be run + throw new IOException( + "Backup merge operation failed, run backup repair tool to restore system's integrity", + e); + } + } finally { + table.close(); + conn.close(); + } + } + + private void checkFailure(FailurePhase phase) throws IOException { + if (failurePhase != null && failurePhase == phase) { + throw new IOException(phase.toString()); + } + } + } + + @Test + public void TestIncBackupMergeRestore() throws Exception { + int ADD_ROWS = 99; + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + // Set custom Merge Job implementation + conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS, + BackupMergeJobWithFailures.class, BackupMergeJob.class); + + Connection conn = ConnectionFactory.createConnection(conf1); + + Admin admin = conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table1 + Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); + LOG.debug("writing " + ADD_ROWS + " rows to " + table1); + + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS); + t1.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table1); + + Table t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS); + + Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS); + t2.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table2); + + // #3 - incremental backup for multiple tables + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdIncMultiple)); + + t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS); + t1.close(); + + t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS); + t2.close(); + + // #3 - incremental backup for multiple tables + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple2)); + // #4 Merge backup images with failures + + for (FailurePhase phase : FailurePhase.values()) { + Configuration conf = conn.getConfiguration(); + + conf.set(FAILURE_PHASE_KEY, phase.toString()); + + try (BackupAdmin bAdmin = new BackupAdminImpl(conn)) { + String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 }; + bAdmin.mergeBackups(backups); + Assert.fail("Expected IOException"); + } catch (IOException e) { + BackupSystemTable table = new BackupSystemTable(conn); + if (phase.ordinal() < FailurePhase.PHASE4.ordinal()) { + // No need to repair: + // Both Merge and backup exclusive operations are finished + assertFalse(table.isMergeInProgress()); + try { + table.finishBackupExclusiveOperation(); + Assert.fail("IOException is expected"); + } catch (IOException ee) { + // Expected + } + } else { + // Repair is required + assertTrue(table.isMergeInProgress()); + try { + table.startBackupExclusiveOperation(); + Assert.fail("IOException is expected"); + } catch (IOException ee) { + // Expected - clean up before proceeding + // table.finishMergeOperation(); + // table.finishBackupExclusiveOperation(); + } + } + table.close(); + LOG.debug("Expected :" + e.getMessage()); + } + } + // Now merge w/o failures + Configuration conf = conn.getConfiguration(); + conf.unset(FAILURE_PHASE_KEY); + conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS); + // Now run repair + BackupSystemTable sysTable = new BackupSystemTable(conn); + BackupCommands.RepairCommand.repairFailedBackupMergeIfAny(conn, sysTable); + // Now repeat merge + try (BackupAdmin bAdmin = new BackupAdminImpl(conn)) { + String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 }; + bAdmin.mergeBackups(backups); + } + + // #6 - restore incremental backup for multiple tables, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2, false, + tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + + Table hTable = conn.getTable(table1_restore); + LOG.debug("After incremental restore: " + hTable.getDescriptor()); + LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows"); + Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH + 2 * ADD_ROWS); + + hTable.close(); + + hTable = conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS); + hTable.close(); + + admin.close(); + conn.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java new file mode 100644 index 000000000000..c8d536564188 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java @@ -0,0 +1,165 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient; +import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +@RunWith(Parameterized.class) +public class TestIncrementalBackupWithFailures extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementalBackupWithFailures.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestIncrementalBackupWithFailures.class); + + @Parameterized.Parameters + public static Collection data() { + provider = "multiwal"; + List params = new ArrayList(); + params.add(new Object[] { Boolean.TRUE }); + return params; + } + + public TestIncrementalBackupWithFailures(Boolean b) { + } + + // implement all test cases in 1 test since incremental backup/restore has dependencies + @Test + public void testIncBackupRestore() throws Exception { + int ADD_ROWS = 99; + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + final byte[] fam3Name = Bytes.toBytes("f3"); + TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name)).build(); + TEST_UTIL.getAdmin().modifyTable(newTable1Desc); + + Connection conn = ConnectionFactory.createConnection(conf1); + int NB_ROWS_FAM3 = 6; + insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close(); + + Admin admin = conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table + Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); + LOG.debug("writing " + ADD_ROWS + " rows to " + table1); + + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3); + t1.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table1); + + Table t2 = conn.getTable(table2); + Put p2; + for (int i = 0; i < 5; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + + Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + 5); + t2.close(); + LOG.debug("written " + 5 + " rows to " + table2); + + // #3 - incremental backup for multiple tables + incrementalBackupWithFailures(); + + admin.close(); + conn.close(); + + } + + private void incrementalBackupWithFailures() throws Exception { + conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS, + IncrementalTableBackupClientForTest.class.getName()); + int maxStage = Stage.values().length - 1; + // Fail stages between 0 and 4 inclusive + for (int stage = 0; stage <= maxStage; stage++) { + LOG.info("Running stage " + stage); + runBackupAndFailAtStage(stage); + } + } + + private void runBackupAndFailAtStage(int stage) throws Exception { + + conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage); + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + String[] args = new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() + "," + table2.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertFalse(ret == 0); + List backups = table.getBackupHistory(); + int after = table.getBackupHistory().size(); + + assertTrue(after == before + 1); + for (BackupInfo data : backups) { + if (data.getType() == BackupType.FULL) { + assertTrue(data.getState() == BackupState.COMPLETE); + } else { + assertTrue(data.getState() == BackupState.FAILED); + } + } + } + } + +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java new file mode 100644 index 000000000000..f9b9ab7ab500 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java @@ -0,0 +1,153 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestRemoteBackup extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoteBackup.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestRemoteBackup.class); + + /** + * Setup Cluster with appropriate configurations before running tests. + * @throws Exception if starting the mini cluster or setting up the tables fails + */ + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + conf1 = TEST_UTIL.getConfiguration(); + conf1.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 10); + useSecondCluster = true; + setUpHelper(); + } + + /** + * Verify that a remote full backup is created on a single table with data correctly. + * @throws Exception if an operation on the table fails + */ + @Test + public void testFullBackupRemote() throws Exception { + LOG.info("test remote full backup on a single table"); + final CountDownLatch latch = new CountDownLatch(1); + final int NB_ROWS_IN_FAM3 = 6; + final byte[] fam3Name = Bytes.toBytes("f3"); + final byte[] fam2Name = Bytes.toBytes("f2"); + final Connection conn = ConnectionFactory.createConnection(conf1); + Thread t = new Thread(() -> { + try { + latch.await(); + } catch (InterruptedException ie) { + } + try { + Table t1 = conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_FAM3; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(fam3Name, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + LOG.debug("Wrote " + NB_ROWS_IN_FAM3 + " rows into family3"); + t1.close(); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + }); + t.start(); + // family 2 is MOB enabled + TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name)) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2Name).setMobEnabled(true) + .setMobThreshold(0L).build()) + .build(); + TEST_UTIL.getAdmin().modifyTable(newTable1Desc); + + SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name); + Table t1 = conn.getTable(table1); + int rows0 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name); + + latch.countDown(); + String backupId = + backupTables(BackupType.FULL, Lists.newArrayList(table1), BACKUP_REMOTE_ROOT_DIR); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete " + backupId); + Assert.assertEquals(TEST_UTIL.countRows(t1, famName), NB_ROWS_IN_BATCH); + + t.join(); + Assert.assertEquals(TEST_UTIL.countRows(t1, fam3Name), NB_ROWS_IN_FAM3); + t1.close(); + + TableName[] tablesRestoreFull = new TableName[] { table1 }; + + TableName[] tablesMapFull = new TableName[] { table1_restore }; + + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, + tablesRestoreFull, tablesMapFull, false)); + + // check tables for full restore + Admin hAdmin = TEST_UTIL.getAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + + // #5.2 - checking row count of tables for full restore + Table hTable = conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH); + int cnt3 = TEST_UTIL.countRows(hTable, fam3Name); + Assert.assertTrue(cnt3 >= 0 && cnt3 <= NB_ROWS_IN_FAM3); + + int rows1 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name); + Assert.assertEquals(rows0, rows1); + hTable.close(); + + hAdmin.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java new file mode 100644 index 000000000000..f2e2b612b2ca --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java @@ -0,0 +1,75 @@ +/* + * 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.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestRemoteRestore extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRemoteRestore.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestRemoteRestore.class); + + /** + * Setup Cluster with appropriate configurations before running tests. + * @throws Exception if starting the mini cluster or setting up the tables fails + */ + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + conf1 = TEST_UTIL.getConfiguration(); + useSecondCluster = true; + setUpHelper(); + } + + /** + * Verify that a remote restore on a single table is successful. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullRestoreRemote() throws Exception { + LOG.info("test remote full backup on a single table"); + String backupId = + backupTables(BackupType.FULL, toList(table1.getNameAsString()), BACKUP_REMOTE_ROOT_DIR); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + getBackupAdmin().restore(BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, + false, tableset, tablemap, false)); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + hba.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java new file mode 100644 index 000000000000..93345fd17059 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java @@ -0,0 +1,95 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestRepairAfterFailedDelete extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRepairAfterFailedDelete.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestRepairAfterFailedDelete.class); + + @Test + public void testRepairBackupDelete() throws Exception { + LOG.info("test repair backup delete on a single table with data"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + String[] backupIds = new String[] { backupId }; + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo info = table.readBackupInfo(backupId); + Path path = new Path(info.getBackupRootDir(), backupId); + FileSystem fs = FileSystem.get(path.toUri(), conf1); + assertTrue(fs.exists(path)); + + // Snapshot backup system table before delete + String snapshotName = "snapshot-backup"; + Connection conn = TEST_UTIL.getConnection(); + Admin admin = conn.getAdmin(); + admin.snapshot(snapshotName, BackupSystemTable.getTableName(conf1)); + + int deleted = getBackupAdmin().deleteBackups(backupIds); + + assertTrue(!fs.exists(path)); + assertTrue(fs.exists(new Path(info.getBackupRootDir()))); + assertTrue(1 == deleted); + + // Emulate delete failure + // Restore backup system table + admin.disableTable(BackupSystemTable.getTableName(conf1)); + admin.restoreSnapshot(snapshotName); + admin.enableTable(BackupSystemTable.getTableName(conf1)); + // Start backup session + table.startBackupExclusiveOperation(); + // Start delete operation + table.startDeleteOperation(backupIds); + + // Now run repair command to repair "failed" delete operation + String[] args = new String[] { "repair" }; + // Run restore + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + // Verify that history length == 0 + assertTrue(table.getBackupHistory().size() == 0); + table.close(); + admin.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java new file mode 100644 index 000000000000..7b49558031e8 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java @@ -0,0 +1,82 @@ +/* + * 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.junit.Assert.assertTrue; + +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestRestoreBoundaryTests extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRestoreBoundaryTests.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestRestoreBoundaryTests.class); + + /** + * Verify that a single empty table is restored to a new table. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullRestoreSingleEmpty() throws Exception { + LOG.info("test full restore on a single table empty table"); + String backupId = fullTableBackup(toList(table1.getNameAsString())); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + getBackupAdmin().restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + tableset, tablemap, false)); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception if doing the backup or an operation on the tables fails + */ + @Test + public void testFullRestoreMultipleEmpty() throws Exception { + LOG.info("create full backup image on multiple tables"); + + List tables = toList(table2.getNameAsString(), table3.getNameAsString()); + String backupId = fullTableBackup(tables); + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + getBackupAdmin().restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, tablemap, false)); + Admin hba = TEST_UTIL.getAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java new file mode 100644 index 000000000000..f626dec5875d --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java @@ -0,0 +1,58 @@ +/* + * 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.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestSystemTableSnapshot extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSystemTableSnapshot.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestSystemTableSnapshot.class); + + /** + * Verify backup system table snapshot. + * @throws Exception if an operation on the table fails + */ + // @Test + public void _testBackupRestoreSystemTable() throws Exception { + LOG.info("test snapshot system table"); + + TableName backupSystem = BackupSystemTable.getTableName(conf1); + + Admin hba = TEST_UTIL.getAdmin(); + String snapshotName = "sysTable"; + hba.snapshot(snapshotName, backupSystem); + + hba.disableTable(backupSystem); + hba.restoreSnapshot(snapshotName); + hba.enableTable(backupSystem); + hba.close(); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java new file mode 100644 index 000000000000..2b0f9c0cba5f --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java @@ -0,0 +1,135 @@ +/* + * 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.master; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.TestBackupBase; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupLogCleaner extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupLogCleaner.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestBackupLogCleaner.class); + + // implements all test cases in 1 test since incremental full backup/ + // incremental backup has dependencies + + @Test + public void testBackupLogCleaner() throws Exception { + + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tableSetFullList = Lists.newArrayList(table1, table2, table3, table4); + + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + // Verify that we have no backup sessions yet + assertFalse(systemTable.hasBackupSessions()); + + List walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + BackupLogCleaner cleaner = new BackupLogCleaner(); + cleaner.setConf(TEST_UTIL.getConfiguration()); + Map params = new HashMap<>(); + params.put(HMaster.MASTER, TEST_UTIL.getHBaseCluster().getMaster()); + cleaner.init(params); + cleaner.setConf(TEST_UTIL.getConfiguration()); + + Iterable deletable = cleaner.getDeletableFiles(walFiles); + int size = Iterables.size(deletable); + + // We can delete all files because we do not have yet recorded backup sessions + assertTrue(size == walFiles.size()); + + String backupIdFull = fullTableBackup(tableSetFullList); + assertTrue(checkSucceeded(backupIdFull)); + // Check one more time + deletable = cleaner.getDeletableFiles(walFiles); + // We can delete wal files because they were saved into backup system table table + size = Iterables.size(deletable); + assertTrue(size == walFiles.size()); + + List newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + LOG.debug("WAL list after full backup"); + + // New list of wal files is greater than the previous one, + // because new wal per RS have been opened after full backup + assertTrue(walFiles.size() < newWalFiles.size()); + Connection conn = ConnectionFactory.createConnection(conf1); + // #2 - insert some data to table + Table t1 = conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + + t1.close(); + + Table t2 = conn.getTable(table2); + Put p2; + for (int i = 0; i < 5; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + + t2.close(); + + // #3 - incremental backup for multiple tables + + List tableSetIncList = Lists.newArrayList(table1, table2, table3); + String backupIdIncMultiple = + backupTables(BackupType.INCREMENTAL, tableSetIncList, BACKUP_ROOT_DIR); + assertTrue(checkSucceeded(backupIdIncMultiple)); + deletable = cleaner.getDeletableFiles(newWalFiles); + + assertTrue(Iterables.size(deletable) == newWalFiles.size()); + + conn.close(); + } + } +} diff --git a/hbase-backup/src/test/resources/hbase-site.xml b/hbase-backup/src/test/resources/hbase-site.xml new file mode 100644 index 000000000000..c20d4bc970c6 --- /dev/null +++ b/hbase-backup/src/test/resources/hbase-site.xml @@ -0,0 +1,170 @@ + + + + + + hbase.regionserver.msginterval + 100 + Interval between messages from the RegionServer to HMaster + in milliseconds. Default is 15. Set this value low if you want unit + tests to be responsive. + + + + hbase.server.thread.wakefrequency + 100 + Time to sleep in between searches for work (in milliseconds). + Used as sleep interval by service threads such as hbase:meta scanner and log roller. + + + + hbase.regionserver.compaction.check.period + 1000 + + + hbase.regionserver.flush.check.period + 1000 + + + hbase.defaults.for.version.skip + true + + + hbase.hconnection.threads.keepalivetime + 3 + + + hbase.regionserver.handler.count + 3 + Default is 30 + + + hbase.regionserver.metahandler.count + 3 + Default is 20 + + + hbase.netty.worker.count + 3 + Default is 0 + + + hbase.hconnection.threads.max + 6 + Default is 256 + + + hbase.htable.threads.max + 3 + Default is MAX_INTEGER + + + hbase.region.replica.replication.threads.max + 7 + Default is 256 + + + hbase.rest.threads.max + 5 + Default is 100 + + + hbase.replication.bulkload.copy.maxthreads + 3 + Default is 10 + + + hbase.loadincremental.threads.max + 1 + Default is # of CPUs + + + hbase.hstore.flusher.count + 1 + Default is 2 + + + hbase.oldwals.cleaner.thread.size + 1 + Default is 2 + + + hbase.master.procedure.threads + 5 + Default is at least 16 + + + hbase.procedure.remote.dispatcher.threadpool.size + 3 + Default is 128 + + + hbase.regionserver.executor.closeregion.threads + 1 + Default is 3 + + + hbase.regionserver.executor.openregion.threads + 1 + Default is 3 + + + hbase.regionserver.executor.openpriorityregion.threads + 1 + Default is 3 + + + hbase.storescanner.parallel.seek.threads + 3 + Default is 10 + + + hbase.hfile.compaction.discharger.thread.count + 1 + Default is 10 + + + hbase.regionserver.executor.refresh.peer.threads + 1 + Default is 2 + + + hbase.hregion.open.and.init.threads.max + 3 + Default is 16 or # of Regions + + + hbase.master.handler.count + 7 + Default is 25 + + + hbase.replication.source.maxthreads + + Default is 10 + + + hbase.hconnection.meta.lookup.threads.max + 5 + Default is 128 + + diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml index a7a2af150a5d..0cbb83823d05 100644 --- a/hbase-it/pom.xml +++ b/hbase-it/pom.xml @@ -105,6 +105,10 @@ org.apache.hbase hbase-server + + org.apache.hbase + hbase-backup + org.apache.hbase ${compat.module} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java new file mode 100644 index 000000000000..4326c9852e35 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java @@ -0,0 +1,439 @@ +/* + * 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; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.backup.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.RestoreRequest; +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.chaos.actions.RestartRandomRsExceptMetaAction; +import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; +import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy; +import org.apache.hadoop.hbase.chaos.policies.Policy; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; + +/** + * An integration test to detect regressions in HBASE-7912. Create a table with many regions, load + * data, perform series backup/load operations, then restore and verify data + * @see HBASE-7912 + * @see HBASE-14123 + */ +@Category(IntegrationTests.class) +public class IntegrationTestBackupRestore extends IntegrationTestBase { + private static final String CLASS_NAME = IntegrationTestBackupRestore.class.getSimpleName(); + protected static final Logger LOG = LoggerFactory.getLogger(IntegrationTestBackupRestore.class); + protected static final String NUMBER_OF_TABLES_KEY = "num_tables"; + protected static final String COLUMN_NAME = "f"; + protected static final String REGION_COUNT_KEY = "regions_per_rs"; + protected static final String REGIONSERVER_COUNT_KEY = "region_servers"; + protected static final String ROWS_PER_ITERATION_KEY = "rows_in_iteration"; + protected static final String NUM_ITERATIONS_KEY = "num_iterations"; + protected static final int DEFAULT_REGION_COUNT = 10; + protected static final int DEFAULT_REGIONSERVER_COUNT = 5; + protected static final int DEFAULT_NUMBER_OF_TABLES = 1; + protected static final int DEFAULT_NUM_ITERATIONS = 10; + protected static final int DEFAULT_ROWS_IN_ITERATION = 500000; + protected static final String SLEEP_TIME_KEY = "sleeptime"; + // short default interval because tests don't run very long. + protected static final long SLEEP_TIME_DEFAULT = 50000L; + + protected static int rowsInIteration; + protected static int regionsCountPerServer; + protected static int regionServerCount; + + protected static int numIterations; + protected static int numTables; + protected static TableName[] tableNames; + protected long sleepTime; + protected static Object lock = new Object(); + + private static String BACKUP_ROOT_DIR = "backupIT"; + + @Override + @Before + public void setUp() throws Exception { + util = new IntegrationTestingUtility(); + Configuration conf = util.getConfiguration(); + regionsCountPerServer = conf.getInt(REGION_COUNT_KEY, DEFAULT_REGION_COUNT); + regionServerCount = conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT); + rowsInIteration = conf.getInt(ROWS_PER_ITERATION_KEY, DEFAULT_ROWS_IN_ITERATION); + numIterations = conf.getInt(NUM_ITERATIONS_KEY, DEFAULT_NUM_ITERATIONS); + numTables = conf.getInt(NUMBER_OF_TABLES_KEY, DEFAULT_NUMBER_OF_TABLES); + sleepTime = conf.getLong(SLEEP_TIME_KEY, SLEEP_TIME_DEFAULT); + enableBackup(conf); + LOG.info("Initializing cluster with {} region servers.", regionServerCount); + util.initializeCluster(regionServerCount); + LOG.info("Cluster initialized and ready"); + } + + @After + public void tearDown() throws IOException { + LOG.info("Cleaning up after test."); + if (util.isDistributedCluster()) { + deleteTablesIfAny(); + LOG.info("Cleaning up after test. Deleted tables"); + cleanUpBackupDir(); + } + LOG.info("Restoring cluster."); + util.restoreCluster(); + LOG.info("Cluster restored."); + } + + @Override + public void setUpMonkey() throws Exception { + Policy p = + new PeriodicRandomActionPolicy(sleepTime, new RestartRandomRsExceptMetaAction(sleepTime)); + this.monkey = new PolicyBasedChaosMonkey(util, p); + startMonkey(); + } + + private void deleteTablesIfAny() throws IOException { + for (TableName table : tableNames) { + util.deleteTableIfAny(table); + } + } + + private void createTables() throws Exception { + tableNames = new TableName[numTables]; + for (int i = 0; i < numTables; i++) { + tableNames[i] = TableName.valueOf(CLASS_NAME + ".table." + i); + } + for (TableName table : tableNames) { + createTable(table); + } + } + + private void enableBackup(Configuration conf) { + // Enable backup + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf); + BackupManager.decorateRegionServerConfiguration(conf); + } + + private void cleanUpBackupDir() throws IOException { + FileSystem fs = FileSystem.get(util.getConfiguration()); + fs.delete(new Path(BACKUP_ROOT_DIR), true); + } + + @Test + public void testBackupRestore() throws Exception { + BACKUP_ROOT_DIR = util.getDataTestDirOnTestFS() + Path.SEPARATOR + BACKUP_ROOT_DIR; + createTables(); + runTestMulti(); + } + + private void runTestMulti() throws IOException { + LOG.info("IT backup & restore started"); + Thread[] workers = new Thread[numTables]; + for (int i = 0; i < numTables; i++) { + final TableName table = tableNames[i]; + Runnable r = new Runnable() { + @Override + public void run() { + try { + runTestSingle(table); + } catch (IOException e) { + LOG.error("Failed", e); + Assert.fail(e.getMessage()); + } + } + }; + workers[i] = new Thread(r); + workers[i].start(); + } + // Wait all workers to finish + for (Thread t : workers) { + Uninterruptibles.joinUninterruptibly(t); + } + LOG.info("IT backup & restore finished"); + } + + private void createTable(TableName tableName) throws Exception { + long startTime, endTime; + + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + + TableDescriptor desc = builder.build(); + ColumnFamilyDescriptorBuilder cbuilder = + ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_NAME.getBytes(Charset.defaultCharset())); + ColumnFamilyDescriptor[] columns = new ColumnFamilyDescriptor[] { cbuilder.build() }; + LOG.info("Creating table {} with {} splits.", tableName, + regionsCountPerServer * regionServerCount); + startTime = EnvironmentEdgeManager.currentTime(); + HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(), desc, columns, + regionsCountPerServer); + util.waitTableAvailable(tableName); + endTime = EnvironmentEdgeManager.currentTime(); + LOG.info("Pre-split table created successfully in {}ms.", (endTime - startTime)); + } + + private void loadData(TableName table, int numRows) throws IOException { + Connection conn = util.getConnection(); + // #0- insert some data to a table + Table t1 = conn.getTable(table); + util.loadRandomRows(t1, new byte[] { 'f' }, 100, numRows); + // flush table + conn.getAdmin().flush(TableName.valueOf(table.getName())); + } + + private String backup(BackupRequest request, BackupAdmin client) throws IOException { + String backupId = client.backupTables(request); + return backupId; + } + + private void restore(RestoreRequest request, BackupAdmin client) throws IOException { + client.restore(request); + } + + private void merge(String[] backupIds, BackupAdmin client) throws IOException { + client.mergeBackups(backupIds); + } + + private void runTestSingle(TableName table) throws IOException { + + List backupIds = new ArrayList(); + + try (Connection conn = util.getConnection(); Admin admin = conn.getAdmin(); + BackupAdmin client = new BackupAdminImpl(conn);) { + + // #0- insert some data to table 'table' + loadData(table, rowsInIteration); + + // #1 - create full backup for table first + LOG.info("create full backup image for {}", table); + List tables = Lists.newArrayList(table); + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = builder.withBackupType(BackupType.FULL).withTableList(tables) + .withTargetRootDir(BACKUP_ROOT_DIR).build(); + + String backupIdFull = backup(request, client); + assertTrue(checkSucceeded(backupIdFull)); + + backupIds.add(backupIdFull); + // Now continue with incremental backups + int count = 1; + while (count++ < numIterations) { + + // Load data + loadData(table, rowsInIteration); + // Do incremental backup + builder = new BackupRequest.Builder(); + request = builder.withBackupType(BackupType.INCREMENTAL).withTableList(tables) + .withTargetRootDir(BACKUP_ROOT_DIR).build(); + String backupId = backup(request, client); + assertTrue(checkSucceeded(backupId)); + backupIds.add(backupId); + + // Restore incremental backup for table, with overwrite for previous backup + String previousBackupId = backupIds.get(backupIds.size() - 2); + restoreVerifyTable(conn, client, table, previousBackupId, rowsInIteration * (count - 1)); + // Restore incremental backup for table, with overwrite for last backup + restoreVerifyTable(conn, client, table, backupId, rowsInIteration * count); + } + // Now merge all incremental and restore + String[] incBackupIds = allIncremental(backupIds); + merge(incBackupIds, client); + // Restore last one + String backupId = incBackupIds[incBackupIds.length - 1]; + // restore incremental backup for table, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { table }; + restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tablesRestoreIncMultiple, null, + true), client); + Table hTable = conn.getTable(table); + Assert.assertEquals(util.countRows(hTable), rowsInIteration * numIterations); + hTable.close(); + LOG.info("{} loop {} finished.", Thread.currentThread().getName(), (count - 1)); + } + } + + private void restoreVerifyTable(Connection conn, BackupAdmin client, TableName table, + String backupId, long expectedRows) throws IOException { + + TableName[] tablesRestoreIncMultiple = new TableName[] { table }; + restore( + createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tablesRestoreIncMultiple, null, true), + client); + Table hTable = conn.getTable(table); + Assert.assertEquals(expectedRows, util.countRows(hTable)); + hTable.close(); + } + + private String[] allIncremental(List backupIds) { + int size = backupIds.size(); + backupIds = backupIds.subList(1, size); + String[] arr = new String[size - 1]; + backupIds.toArray(arr); + return arr; + } + + /** + * Check if backup is succeeded + * @param backupId pass backup ID to check status of + * @return status of backup + */ + protected boolean checkSucceeded(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + if (status == null) { + return false; + } + return status.getState() == BackupState.COMPLETE; + } + + private BackupInfo getBackupInfo(String backupId) throws IOException { + try (BackupSystemTable table = new BackupSystemTable(util.getConnection())) { + return table.readBackupInfo(backupId); + } + } + + /** + * Get restore request. + * @param backupRootDir directory where backup is located + * @param backupId backup ID + * @param check check the backup + * @param fromTables table names to restore from + * @param toTables new table names to restore to + * @param isOverwrite overwrite the table(s) + * @return an instance of RestoreRequest + */ + public RestoreRequest createRestoreRequest(String backupRootDir, String backupId, boolean check, + TableName[] fromTables, TableName[] toTables, boolean isOverwrite) { + RestoreRequest.Builder builder = new RestoreRequest.Builder(); + return builder.withBackupRootDir(backupRootDir).withBackupId(backupId).withCheck(check) + .withFromTables(fromTables).withToTables(toTables).withOvewrite(isOverwrite).build(); + } + + @Override + public void setUpCluster() throws Exception { + util = getTestingUtil(getConf()); + enableBackup(getConf()); + LOG.debug("Initializing/checking cluster has {} servers", regionServerCount); + util.initializeCluster(regionServerCount); + LOG.debug("Done initializing/checking cluster"); + } + + /** Returns status of CLI execution */ + @Override + public int runTestFromCommandLine() throws Exception { + // Check if backup is enabled + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + testBackupRestore(); + return 0; + } + + @Override + public TableName getTablename() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected Set getColumnFamilies() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected void addOptions() { + addOptWithArg(REGIONSERVER_COUNT_KEY, + "Total number of region servers. Default: '" + DEFAULT_REGIONSERVER_COUNT + "'"); + addOptWithArg(REGION_COUNT_KEY, "Total number of regions. Default: " + DEFAULT_REGION_COUNT); + addOptWithArg(ROWS_PER_ITERATION_KEY, + "Total number of data rows to be loaded during one iteration." + " Default: " + + DEFAULT_ROWS_IN_ITERATION); + addOptWithArg(NUM_ITERATIONS_KEY, + "Total number iterations." + " Default: " + DEFAULT_NUM_ITERATIONS); + addOptWithArg(NUMBER_OF_TABLES_KEY, + "Total number of tables in the test." + " Default: " + DEFAULT_NUMBER_OF_TABLES); + addOptWithArg(SLEEP_TIME_KEY, "Sleep time of chaos monkey in ms " + + "to restart random region server. Default: " + SLEEP_TIME_DEFAULT); + } + + @Override + protected void processOptions(CommandLine cmd) { + super.processOptions(cmd); + regionsCountPerServer = Integer + .parseInt(cmd.getOptionValue(REGION_COUNT_KEY, Integer.toString(DEFAULT_REGION_COUNT))); + regionServerCount = Integer.parseInt( + cmd.getOptionValue(REGIONSERVER_COUNT_KEY, Integer.toString(DEFAULT_REGIONSERVER_COUNT))); + rowsInIteration = Integer.parseInt( + cmd.getOptionValue(ROWS_PER_ITERATION_KEY, Integer.toString(DEFAULT_ROWS_IN_ITERATION))); + numIterations = Integer + .parseInt(cmd.getOptionValue(NUM_ITERATIONS_KEY, Integer.toString(DEFAULT_NUM_ITERATIONS))); + numTables = Integer.parseInt( + cmd.getOptionValue(NUMBER_OF_TABLES_KEY, Integer.toString(DEFAULT_NUMBER_OF_TABLES))); + sleepTime = + Long.parseLong(cmd.getOptionValue(SLEEP_TIME_KEY, Long.toString(SLEEP_TIME_DEFAULT))); + + LOG.info(MoreObjects.toStringHelper("Parsed Options") + .add(REGION_COUNT_KEY, regionsCountPerServer).add(REGIONSERVER_COUNT_KEY, regionServerCount) + .add(ROWS_PER_ITERATION_KEY, rowsInIteration).add(NUM_ITERATIONS_KEY, numIterations) + .add(NUMBER_OF_TABLES_KEY, numTables).add(SLEEP_TIME_KEY, sleepTime).toString()); + } + + /** + * Main method + * @param args argument list + */ + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + IntegrationTestingUtility.setUseDistributedCluster(conf); + int status = ToolRunner.run(conf, new IntegrationTestBackupRestore(), args); + System.exit(status); + } +} diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index 22ee8727466a..23488ad985f2 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -27,6 +27,7 @@ import java.net.InetSocketAddress; import java.net.URLDecoder; import java.net.URLEncoder; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -153,6 +154,17 @@ protected static byte[] combineTableNameSuffix(byte[] tableName, byte[] suffix) static final String DATABLOCK_ENCODING_FAMILIES_CONF_KEY = "hbase.mapreduce.hfileoutputformat.families.datablock.encoding"; + // When MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY is enabled, should table names be written + // with namespace included. Enabling this means downstream jobs which use this output will + // need to account for namespace when finding the directory of the job output. + // For example: a table named my-table in namespace default would be in `/output/default/my-table` + // instead of current `/output/my-table` + // This will be the behavior when upgrading to hbase 3.0. + public static final String TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY = + "hbase.hfileoutputformat.tablename.namespace.inclusive"; + + private static final boolean TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_DEFAULT_VALUE = false; + // This constant is public since the client can modify this when setting // up their conf object and thus refer to this symbol. // It is present for backwards compatibility reasons. Use it only to @@ -201,6 +213,8 @@ static RecordWriter createRecordWrit final Configuration conf = context.getConfiguration(); final boolean writeMultipleTables = conf.getBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, false); + final boolean writeToTableWithNamespace = conf.getBoolean( + TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY, TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_DEFAULT_VALUE); final String writeTableNames = conf.get(OUTPUT_TABLE_NAME_CONF_KEY); if (writeTableNames == null || writeTableNames.isEmpty()) { throw new IllegalArgumentException("" + OUTPUT_TABLE_NAME_CONF_KEY + " cannot be empty"); @@ -254,7 +268,10 @@ public void write(ImmutableBytesWritable row, V cell) throws IOException { byte[] tableNameBytes = null; if (writeMultipleTables) { tableNameBytes = MultiTableHFileOutputFormat.getTableName(row.get()); - tableNameBytes = TableName.valueOf(tableNameBytes).toBytes(); + tableNameBytes = writeToTableWithNamespace + ? TableName.valueOf(tableNameBytes).getNameWithNamespaceInclAsString() + .getBytes(Charset.defaultCharset()) + : TableName.valueOf(tableNameBytes).toBytes(); if (!allTableNames.contains(Bytes.toString(tableNameBytes))) { throw new IllegalArgumentException( "TableName " + Bytes.toString(tableNameBytes) + " not expected"); @@ -264,6 +281,7 @@ public void write(ImmutableBytesWritable row, V cell) throws IOException { } Path tableRelPath = getTableRelativePath(tableNameBytes); byte[] tableAndFamily = getTableNameSuffixedWithFamily(tableNameBytes, family); + WriterLength wl = this.writers.get(tableAndFamily); // If this is a new column family, verify that the directory exists @@ -290,7 +308,6 @@ public void write(ImmutableBytesWritable row, V cell) throws IOException { if (wl == null || wl.writer == null) { if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) { HRegionLocation loc = null; - String tableName = Bytes.toString(tableNameBytes); if (tableName != null) { try ( @@ -611,6 +628,9 @@ static void configureIncrementalLoad(Job job, List multiTableInfo, job.setOutputValueClass(MapReduceExtendedCell.class); job.setOutputFormatClass(cls); + final boolean writeToTableWithNamespace = conf.getBoolean( + TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY, TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_DEFAULT_VALUE); + if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) { throw new IllegalArgumentException("Duplicate entries found in TableInfo argument"); } @@ -650,7 +670,9 @@ static void configureIncrementalLoad(Job job, List multiTableInfo, for (TableInfo tableInfo : multiTableInfo) { regionLocators.add(tableInfo.getRegionLocator()); - allTableNames.add(tableInfo.getRegionLocator().getName().getNameAsString()); + allTableNames.add(writeMultipleTables && writeToTableWithNamespace + ? tableInfo.getRegionLocator().getName().getNameWithNamespaceInclAsString() + : tableInfo.getRegionLocator().getName().getNameAsString()); tableDescriptors.add(tableInfo.getTableDescriptor()); } // Record tablenames for creating writer by favored nodes, and decoding compression, diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index 9f5658cd8967..63cc3e1604f3 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -20,7 +20,11 @@ import java.io.IOException; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -39,6 +43,7 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.TableInfo; import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -71,6 +76,9 @@ public class WALPlayer extends Configured implements Tool { public final static String TABLE_MAP_KEY = "wal.input.tablesmap"; public final static String INPUT_FILES_SEPARATOR_KEY = "wal.input.separator"; public final static String IGNORE_MISSING_FILES = "wal.input.ignore.missing.files"; + public final static String MULTI_TABLES_SUPPORT = "wal.multi.tables.support"; + + protected static final String tableSeparator = ";"; private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; @@ -88,19 +96,24 @@ protected WALPlayer(final Configuration c) { */ @Deprecated static class WALKeyValueMapper extends Mapper { - private byte[] table; + private Set tableSet = new HashSet(); + private boolean multiTableSupport = false; @Override public void map(WALKey key, WALEdit value, Context context) throws IOException { try { - // skip all other tables - if (Bytes.equals(table, key.getTableName().getName())) { + TableName table = key.getTableName(); + if (tableSet.contains(table.getNameAsString())) { for (Cell cell : value.getCells()) { - KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - if (WALEdit.isMetaEditFamily(kv)) { + if (WALEdit.isMetaEditFamily(cell)) { continue; } - context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv); + KeyValue keyValue = KeyValueUtil.ensureKeyValue(cell); + byte[] outKey = multiTableSupport + ? Bytes.add(table.getName(), Bytes.toBytes(tableSeparator), + CellUtil.cloneRow(keyValue)) + : CellUtil.cloneRow(keyValue); + context.write(new ImmutableBytesWritable(outKey), keyValue); } } } catch (InterruptedException e) { @@ -110,35 +123,35 @@ public void map(WALKey key, WALEdit value, Context context) throws IOException { @Override public void setup(Context context) throws IOException { - // only a single table is supported when HFiles are generated with HFileOutputFormat - String[] tables = context.getConfiguration().getStrings(TABLES_KEY); - if (tables == null || tables.length != 1) { - // this can only happen when WALMapper is used directly by a class other than WALPlayer - throw new IOException("Exactly one table must be specified for bulk HFile case."); + Configuration conf = context.getConfiguration(); + String[] tables = conf.getStrings(TABLES_KEY); + this.multiTableSupport = conf.getBoolean(MULTI_TABLES_SUPPORT, false); + for (String table : tables) { + tableSet.add(table); } - table = Bytes.toBytes(tables[0]); - } - } /** * A mapper that just writes out Cells. This one can be used together with {@link CellSortReducer} */ static class WALCellMapper extends Mapper { - private byte[] table; + private Set tableSet = new HashSet<>(); + private boolean multiTableSupport = false; @Override public void map(WALKey key, WALEdit value, Context context) throws IOException { try { - // skip all other tables - if (Bytes.equals(table, key.getTableName().getName())) { + TableName table = key.getTableName(); + if (tableSet.contains(table.getNameAsString())) { for (Cell cell : value.getCells()) { if (WALEdit.isMetaEditFamily(cell)) { continue; } - context.write(new ImmutableBytesWritable(CellUtil.cloneRow(cell)), - new MapReduceExtendedCell(cell)); + byte[] outKey = multiTableSupport + ? Bytes.add(table.getName(), Bytes.toBytes(tableSeparator), CellUtil.cloneRow(cell)) + : CellUtil.cloneRow(cell); + context.write(new ImmutableBytesWritable(outKey), new MapReduceExtendedCell(cell)); } } } catch (InterruptedException e) { @@ -148,16 +161,13 @@ public void map(WALKey key, WALEdit value, Context context) throws IOException { @Override public void setup(Context context) throws IOException { - // only a single table is supported when HFiles are generated with HFileOutputFormat - String[] tables = context.getConfiguration().getStrings(TABLES_KEY); - if (tables == null || tables.length != 1) { - // this can only happen when WALMapper is used directly by a class other than WALPlayer - throw new IOException("Exactly one table must be specified for bulk HFile case."); + Configuration conf = context.getConfiguration(); + String[] tables = conf.getStrings(TABLES_KEY); + this.multiTableSupport = conf.getBoolean(MULTI_TABLES_SUPPORT, false); + for (String table : tables) { + tableSet.add(table); } - table = Bytes.toBytes(tables[0]); - } - } /** @@ -325,6 +335,8 @@ public Job createSubmittableJob(String[] args) throws IOException { // if no mapping is specified, map each table to itself tableMap = tables; } + + boolean multiTableSupport = conf.getBoolean(MULTI_TABLES_SUPPORT, false); conf.setStrings(TABLES_KEY, tables); conf.setStrings(TABLE_MAP_KEY, tableMap); conf.set(FileInputFormat.INPUT_DIR, inputDirs); @@ -339,20 +351,32 @@ public Job createSubmittableJob(String[] args) throws IOException { if (hfileOutPath != null) { LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs); - // the bulk HFile case - if (tables.length != 1) { + if (!multiTableSupport && tables.length != 1) { throw new IOException("Exactly one table must be specified for the bulk export option"); } - TableName tableName = TableName.valueOf(tables[0]); + + // the bulk HFile case + List tableNames = getTableNameList(tables); + job.setMapperClass(WALCellMapper.class); job.setReducerClass(CellSortReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); job.setMapOutputValueClass(MapReduceExtendedCell.class); - try (Connection conn = ConnectionFactory.createConnection(conf); - Table table = conn.getTable(tableName); - RegionLocator regionLocator = conn.getRegionLocator(tableName)) { - HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); + try (Connection conn = ConnectionFactory.createConnection(conf);) { + List tableInfoList = new ArrayList<>(); + for (TableName tableName : tableNames) { + Table table = conn.getTable(tableName); + RegionLocator regionLocator = conn.getRegionLocator(tableName); + tableInfoList.add(new TableInfo(table.getDescriptor(), regionLocator)); + } + if (multiTableSupport) { + MultiTableHFileOutputFormat.configureIncrementalLoad(job, tableInfoList); + } else { + TableInfo tableInfo = tableInfoList.get(0); + HFileOutputFormat2.configureIncrementalLoad(job, tableInfo.getTableDescriptor(), + tableInfo.getRegionLocator()); + } } TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class); @@ -375,6 +399,14 @@ public Job createSubmittableJob(String[] args) throws IOException { return job; } + private List getTableNameList(String[] tables) { + List list = new ArrayList(); + for (String name : tables) { + list.add(TableName.valueOf(name)); + } + return list; + } + /** * Print usage * @param errorMsg Error message. Can be null. diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java index 9b207158817c..f535ccf7e3e2 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java @@ -566,22 +566,33 @@ public void testMRIncrementalLoadWithPutSortReducer() throws Exception { private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, boolean putSortReducer, String tableStr) throws Exception { - doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer, + doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer, false, Arrays.asList(tableStr)); } @Test public void testMultiMRIncrementalLoadWithPutSortReducer() throws Exception { LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducer\n"); - doIncrementalLoadTest(false, false, true, + doIncrementalLoadTest(false, false, true, false, + Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList())); + } + + @Test + public void testMultiMRIncrementalLoadWithPutSortReducerWithNamespaceInPath() throws Exception { + LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducerWithNamespaceInPath\n"); + doIncrementalLoadTest(false, false, true, true, Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList())); } private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, - boolean putSortReducer, List tableStr) throws Exception { + boolean putSortReducer, boolean shouldWriteToTableWithNamespace, List tableStr) + throws Exception { util = new HBaseTestingUtility(); Configuration conf = util.getConfiguration(); conf.setBoolean(MultiTableHFileOutputFormat.LOCALITY_SENSITIVE_CONF_KEY, shouldKeepLocality); + if (shouldWriteToTableWithNamespace) { + conf.setBoolean(HFileOutputFormat2.TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY, true); + } int hostCount = 1; int regionNum = 5; if (shouldKeepLocality) { @@ -616,13 +627,17 @@ private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKe Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad"); // Generate the bulk load files runIncrementalPELoad(conf, tableInfo, testDir, putSortReducer); + if (shouldWriteToTableWithNamespace) { + testDir = new Path(testDir, "default"); + } for (Table tableSingle : allTables.values()) { // This doesn't write into the table, just makes files assertEquals("HFOF should not touch actual table", 0, util.countRows(tableSingle)); } int numTableDirs = 0; - for (FileStatus tf : testDir.getFileSystem(conf).listStatus(testDir)) { + FileStatus[] fss = testDir.getFileSystem(conf).listStatus(testDir); + for (FileStatus tf : fss) { Path tablePath = testDir; if (writeMultipleTables) { @@ -636,7 +651,8 @@ private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKe // Make sure that a directory was created for every CF int dir = 0; - for (FileStatus f : tablePath.getFileSystem(conf).listStatus(tablePath)) { + fss = tablePath.getFileSystem(conf).listStatus(tablePath); + for (FileStatus f : fss) { for (byte[] family : FAMILIES) { if (Bytes.toString(family).equals(f.getPath().getName())) { ++dir; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 99b27f05630b..e0c596e14cad 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -599,22 +599,33 @@ public void testMRIncrementalLoadWithPutSortReducer() throws Exception { private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, boolean putSortReducer, String tableStr) throws Exception { - doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer, + doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer, false, Arrays.asList(tableStr)); } @Test public void testMultiMRIncrementalLoadWithPutSortReducer() throws Exception { LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducer\n"); - doIncrementalLoadTest(false, false, true, + doIncrementalLoadTest(false, false, true, false, + Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList())); + } + + @Test + public void testMultiMRIncrementalLoadWithPutSortReducerWithNamespaceInPath() throws Exception { + LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducerWithNamespaceInPath\n"); + doIncrementalLoadTest(false, false, true, true, Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList())); } private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, - boolean putSortReducer, List tableStr) throws Exception { + boolean putSortReducer, boolean shouldWriteToTableWithNamespace, List tableStr) + throws Exception { util = new HBaseTestingUtility(); Configuration conf = util.getConfiguration(); conf.setBoolean(MultiTableHFileOutputFormat.LOCALITY_SENSITIVE_CONF_KEY, shouldKeepLocality); + if (shouldWriteToTableWithNamespace) { + conf.setBoolean(HFileOutputFormat2.TABLE_NAME_WITH_NAMESPACE_INCLUSIVE_KEY, true); + } int hostCount = 1; int regionNum = 5; if (shouldKeepLocality) { @@ -651,15 +662,18 @@ private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKe Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad"); // Generate the bulk load files runIncrementalPELoad(conf, tableInfo, testDir, putSortReducer); + if (shouldWriteToTableWithNamespace) { + testDir = new Path(testDir, "default"); + } for (Table tableSingle : allTables.values()) { // This doesn't write into the table, just makes files assertEquals("HFOF should not touch actual table", 0, util.countRows(tableSingle)); } int numTableDirs = 0; - for (FileStatus tf : testDir.getFileSystem(conf).listStatus(testDir)) { + FileStatus[] fss = testDir.getFileSystem(conf).listStatus(testDir); + for (FileStatus tf : fss) { Path tablePath = testDir; - if (writeMultipleTables) { if (allTables.containsKey(tf.getPath().getName())) { ++numTableDirs; @@ -671,7 +685,8 @@ private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKe // Make sure that a directory was created for every CF int dir = 0; - for (FileStatus f : tablePath.getFileSystem(conf).listStatus(tablePath)) { + fss = tablePath.getFileSystem(conf).listStatus(tablePath); + for (FileStatus f : fss) { for (byte[] family : FAMILIES) { if (Bytes.toString(family).equals(f.getPath().getName())) { ++dir; diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto new file mode 100644 index 000000000000..afe43122f848 --- /dev/null +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -0,0 +1,121 @@ +/** + * 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 optional 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. + */ +syntax = "proto2"; +// This file contains Backup manifest +package hbase.pb; + +option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated"; +option java_outer_classname = "BackupProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +import "HBase.proto"; + +/** + * Backup type enum: FULL or INCREMENTAL + */ + +enum BackupType { + FULL = 0; + INCREMENTAL = 1; +} +/** + * ServerTimestamp keeps last WAL roll time per Region Server + */ +message ServerTimestamp { + optional ServerName server_name = 1; + optional uint64 timestamp = 2; +} + +/** + * TableServerTimestamp keeps last WAL roll time per Region Server & Table + * Each table have different last WAL roll time stamps across cluster, on every RS + */ +message TableServerTimestamp { + optional TableName table_name = 1; + repeated ServerTimestamp server_timestamp = 2; +} + +/** + * Structure keeps relevant info for backup restore session + */ +message BackupImage { + optional string backup_id = 1; + optional BackupType backup_type = 2; + optional string backup_root_dir = 3; + repeated TableName table_list = 4; + optional uint64 start_ts = 5; + optional uint64 complete_ts = 6; + repeated BackupImage ancestors = 7; + repeated TableServerTimestamp tst_map = 8; + +} + +/** + * Internal structure used during backup + */ +message BackupTableInfo { + optional TableName table_name = 1; + optional string snapshot_name = 2; +} + +/** + * Backup session information + */ +message BackupInfo { + optional string backup_id = 1; + optional BackupType backup_type = 2; + optional string backup_root_dir = 3; + optional BackupState backup_state = 4; + optional BackupPhase backup_phase = 5; + optional string failed_message = 6; + repeated BackupTableInfo backup_table_info = 7; + optional uint64 start_ts = 8; + optional uint64 complete_ts = 9; + optional uint32 progress = 10; + optional uint32 workers_number = 11; + optional uint64 bandwidth = 12; + map table_set_timestamp = 13; + + message RSTimestampMap { + map rs_timestamp = 1; + } + /** + * Backup session states + */ + enum BackupState { + RUNNING = 0; + COMPLETE = 1; + FAILED = 2; + CANCELLED = 3; + } + + /** + * Phases of a backup session in RUNNING state + * + */ + enum BackupPhase { + REQUEST = 0; + SNAPSHOT = 1; + PREPARE_INCREMENTAL = 2; + SNAPSHOTCOPY = 3; + INCREMENTAL_COPY = 4; + STORE_MANIFEST = 5; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinationManager.java new file mode 100644 index 000000000000..6761cd3a1dd2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinationManager.java @@ -0,0 +1,35 @@ +/* + * 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.procedure; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; + +@InterfaceAudience.Private +public interface ProcedureCoordinationManager { + + /** + * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs} + */ + ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode); + + /** + * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs} + */ + ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws KeeperException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinationManager.java new file mode 100644 index 000000000000..ff0fc5770c41 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinationManager.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.procedure; + +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; + +@InterfaceAudience.Private +public class ZKProcedureCoordinationManager implements ProcedureCoordinationManager { + ZKWatcher watcher; + + public ZKProcedureCoordinationManager(Server server) { + this.watcher = server.getZooKeeper(); + } + + @Override + public ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode) { + return new ZKProcedureCoordinator(watcher, procType, coordNode); + } + + @Override + public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws KeeperException { + return new ZKProcedureMemberRpcs(watcher, procType); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 38300df5568e..c60ba93487ff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -86,7 +86,7 @@ public class FSTableDescriptors implements TableDescriptors { /** * The file name prefix used to store HTD in HDFS */ - static final String TABLEINFO_FILE_PREFIX = ".tableinfo"; + public static final String TABLEINFO_FILE_PREFIX = ".tableinfo"; public static final String TABLEINFO_DIR = ".tabledesc"; diff --git a/pom.xml b/pom.xml index 1a6a62a8224d..bbaf0dd5aa0d 100644 --- a/pom.xml +++ b/pom.xml @@ -500,6 +500,7 @@ hbase-asyncfs hbase-logging hbase-compression + hbase-backup scm:git:git://gitbox.apache.org/repos/asf/hbase.git @@ -850,6 +851,11 @@ test-jar test + + org.apache.hbase + hbase-backup + ${project.version} + org.apache.hbase hbase-mapreduce