From de69f0df34a6cb00a9fd2e370fdf898274871294 Mon Sep 17 00:00:00 2001 From: tedyu Date: Sun, 27 Dec 2015 10:02:09 -0800 Subject: [PATCH] HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov) --- bin/hbase | 6 + .../org/apache/hadoop/hbase/HConstants.java | 443 +++++----- hbase-server/pom.xml | 10 + .../hadoop/hbase/backup/BackupClient.java | 338 ++++++++ .../hadoop/hbase/backup/BackupCommands.java | 147 ++++ .../hadoop/hbase/backup/BackupContext.java | 318 +++++++ .../hbase/backup/BackupCopyService.java | 37 + .../hadoop/hbase/backup/BackupException.java | 85 ++ .../hadoop/hbase/backup/BackupHandler.java | 744 ++++++++++++++++ .../hadoop/hbase/backup/BackupManager.java | 488 +++++++++++ .../hadoop/hbase/backup/BackupManifest.java | 814 ++++++++++++++++++ .../hbase/backup/BackupRestoreConstants.java | 66 ++ .../backup/BackupRestoreServiceFactory.java | 61 ++ .../hadoop/hbase/backup/BackupStatus.java | 67 ++ .../hbase/backup/BackupSystemTable.java | 642 ++++++++++++++ .../hbase/backup/BackupSystemTableHelper.java | 314 +++++++ .../hadoop/hbase/backup/BackupUtil.java | 564 ++++++++++++ .../hbase/backup/HBackupFileSystem.java | 511 +++++++++++ .../backup/IncrementalBackupManager.java | 269 ++++++ .../backup/IncrementalRestoreService.java | 33 + .../hadoop/hbase/backup/RestoreClient.java | 496 +++++++++++ .../hadoop/hbase/backup/RestoreUtil.java | 503 +++++++++++ .../mapreduce/MapReduceBackupCopyService.java | 292 +++++++ .../mapreduce/MapReduceRestoreService.java | 72 ++ .../hbase/backup/master/BackupLogCleaner.java | 121 +++ .../master/LogRollMasterProcedureManager.java | 129 +++ .../LogRollBackupSubprocedure.java | 138 +++ .../LogRollBackupSubprocedurePool.java | 137 +++ .../LogRollRegionServerProcedureManager.java | 168 ++++ .../BaseCoordinatedStateManager.java | 17 + .../ZkCoordinatedStateManager.java | 20 +- .../hadoop/hbase/mapreduce/WALPlayer.java | 54 +- .../apache/hadoop/hbase/master/HMaster.java | 2 + .../RegionServerProcedureManager.java | 2 +- .../RegionServerProcedureManagerHost.java | 3 +- .../procedure/ZKProcedureCoordinatorRpcs.java | 2 +- .../procedure/ZKProcedureMemberRpcs.java | 85 +- ...egionServerFlushTableProcedureManager.java | 2 +- .../hbase/regionserver/HRegionServer.java | 4 +- .../snapshot/RegionServerSnapshotManager.java | 2 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 67 +- .../hadoop/hbase/snapshot/SnapshotCopy.java | 42 + .../hadoop/hbase/wal/DefaultWALProvider.java | 7 +- .../hadoop/hbase/backup/TestBackupBase.java | 194 +++++ .../hbase/backup/TestBackupBoundaryTests.java | 99 +++ .../hbase/backup/TestBackupLogCleaner.java | 161 ++++ .../hbase/backup/TestBackupSystemTable.java | 341 ++++++++ .../hadoop/hbase/backup/TestFullBackup.java | 83 ++ .../hadoop/hbase/backup/TestFullRestore.java | 166 ++++ .../hbase/backup/TestIncrementalBackup.java | 179 ++++ .../hadoop/hbase/backup/TestRemoteBackup.java | 44 + .../hbase/backup/TestRemoteRestore.java | 54 ++ .../backup/TestRestoreBoundaryTests.java | 85 ++ .../procedure/SimpleRSProcedureManager.java | 2 +- 54 files changed, 9389 insertions(+), 341 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java diff --git a/bin/hbase b/bin/hbase index 5064451b4d0..9fd903ad555 100755 --- a/bin/hbase +++ b/bin/hbase @@ -99,6 +99,8 @@ if [ $# = 0 ]; then echo " pe Run PerformanceEvaluation" echo " ltt Run LoadTestTool" echo " version Print the version" + echo " backup backup tables for recovery" + echo " restore restore tables from existing backup image" echo " CLASSNAME Run the class named CLASSNAME" exit 1 fi @@ -303,6 +305,10 @@ elif [ "$COMMAND" = "hfile" ] ; then CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter' elif [ "$COMMAND" = "zkcli" ] ; then CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer" +elif [ "$COMMAND" = "backup" ] ; then + CLASS='org.apache.hadoop.hbase.backup.BackupClient' +elif [ "$COMMAND" = "restore" ] ; then + CLASS='org.apache.hadoop.hbase.backup.RestoreClient' elif [ "$COMMAND" = "upgrade" ] ; then echo "This command was used to upgrade to HBase 0.96, it was removed in HBase 2.0.0." echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading." diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 6fafad37966..0f572e6c04b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -40,11 +40,11 @@ import org.apache.hadoop.hbase.util.Bytes; public final class HConstants { // NOTICE!!!! Please do not add a constants here, unless they are referenced by a lot of classes. - //Bytes.UTF8_ENCODING should be updated if this changed + // Bytes.UTF8_ENCODING should be updated if this changed /** When we encode strings, we always specify UTF8 encoding */ public static final String UTF8_ENCODING = "UTF-8"; - //Bytes.UTF8_CHARSET should be updated if this changed + // Bytes.UTF8_CHARSET should be updated if this changed /** When we encode strings, we always specify UTF8 encoding */ public static final Charset UTF8_CHARSET = Charset.forName(UTF8_ENCODING); /** @@ -55,9 +55,9 @@ public final class HConstants { /** Used as a magic return value while optimized index key feature enabled(HBASE-7845) */ public final static int INDEX_KEY_MAGIC = -2; /* - * Name of directory that holds recovered edits written by the wal log - * splitting code, one per region - */ + * Name of directory that holds recovered edits written by the wal log splitting code, one per + * region + */ public static final String RECOVERED_EDITS_DIR = "recovered.edits"; /** * The first four bytes of Hadoop RPC connections @@ -70,27 +70,24 @@ public final class HConstants { /** The size data structures with minor version is 0 */ public static final int HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG; - /** The size of a version 2 HFile block header, minor version 1. - * There is a 1 byte checksum type, followed by a 4 byte bytesPerChecksum - * followed by another 4 byte value to store sizeofDataOnDisk. + /** + * The size of a version 2 HFile block header, minor version 1. There is a 1 byte checksum type, + * followed by a 4 byte bytesPerChecksum followed by another 4 byte value to store + * sizeofDataOnDisk. */ - public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + - Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT; + public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM + + Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT; /** Just an array of bytes of the right size. */ public static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HFILEBLOCK_HEADER_SIZE]; - //End HFileBlockConstants. + // End HFileBlockConstants. /** * Status codes used for return values of bulk operations. */ @InterfaceAudience.Private public enum OperationStatusCode { - NOT_RUN, - SUCCESS, - BAD_FAMILY, - SANITY_CHECK_FAILURE, - FAILURE; + NOT_RUN, SUCCESS, BAD_FAMILY, SANITY_CHECK_FAILURE, FAILURE; } /** long constant for zero */ @@ -104,19 +101,16 @@ public final class HConstants { public static final String VERSION_FILE_NAME = "hbase.version"; /** - * Current version of file system. - * Version 4 supports only one kind of bloom filter. - * Version 5 changes versions in catalog table regions. - * Version 6 enables blockcaching on catalog tables. - * Version 7 introduces hfile -- hbase 0.19 to 0.20.. - * Version 8 introduces namespace + * Current version of file system. Version 4 supports only one kind of bloom filter. Version 5 + * changes versions in catalog table regions. Version 6 enables blockcaching on catalog tables. + * Version 7 introduces hfile -- hbase 0.19 to 0.20.. Version 8 introduces namespace */ // public static final String FILE_SYSTEM_VERSION = "6"; public static final String FILE_SYSTEM_VERSION = "8"; // Configuration parameters - //TODO: Is having HBase homed on port 60k OK? + // TODO: Is having HBase homed on port 60k OK? /** Cluster is in distributed mode or not */ public static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed"; @@ -131,12 +125,10 @@ public final class HConstants { public static final String ENSEMBLE_TABLE_NAME = "hbase:ensemble"; /** Config for pluggable region normalizer */ - public static final String HBASE_MASTER_NORMALIZER_CLASS = - "hbase.master.normalizer.class"; + public static final String HBASE_MASTER_NORMALIZER_CLASS = "hbase.master.normalizer.class"; /** Config for enabling/disabling pluggable region normalizer */ - public static final String HBASE_NORMALIZER_ENABLED = - "hbase.normalizer.enabled"; + public static final String HBASE_NORMALIZER_ENABLED = "hbase.normalizer.enabled"; /** Cluster is standalone or pseudo-distributed */ public static final boolean CLUSTER_IS_LOCAL = false; @@ -174,21 +166,18 @@ public final class HConstants { public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum"; /** Common prefix of ZooKeeper configuration properties */ - public static final String ZK_CFG_PROPERTY_PREFIX = - "hbase.zookeeper.property."; + public static final String ZK_CFG_PROPERTY_PREFIX = "hbase.zookeeper.property."; - public static final int ZK_CFG_PROPERTY_PREFIX_LEN = - ZK_CFG_PROPERTY_PREFIX.length(); + public static final int ZK_CFG_PROPERTY_PREFIX_LEN = ZK_CFG_PROPERTY_PREFIX.length(); /** - * The ZK client port key in the ZK properties map. The name reflects the - * fact that this is not an HBase configuration key. + * The ZK client port key in the ZK properties map. The name reflects the fact that this is not an + * HBase configuration key. */ public static final String CLIENT_PORT_STR = "clientPort"; /** Parameter name for the client port that the zookeeper listens on */ - public static final String ZOOKEEPER_CLIENT_PORT = - ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR; + public static final String ZOOKEEPER_CLIENT_PORT = ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR; /** Default client port that the zookeeper listens on */ public static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181; @@ -208,19 +197,15 @@ public final class HConstants { public static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase"; /** - * Parameter name for the limit on concurrent client-side zookeeper - * connections + * Parameter name for the limit on concurrent client-side zookeeper connections */ - public static final String ZOOKEEPER_MAX_CLIENT_CNXNS = - ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns"; + public static final String ZOOKEEPER_MAX_CLIENT_CNXNS = ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns"; /** Parameter name for the ZK data directory */ - public static final String ZOOKEEPER_DATA_DIR = - ZK_CFG_PROPERTY_PREFIX + "dataDir"; + public static final String ZOOKEEPER_DATA_DIR = ZK_CFG_PROPERTY_PREFIX + "dataDir"; /** Parameter name for the ZK tick time */ - public static final String ZOOKEEPER_TICK_TIME = - ZK_CFG_PROPERTY_PREFIX + "tickTime"; + public static final String ZOOKEEPER_TICK_TIME = ZK_CFG_PROPERTY_PREFIX + "tickTime"; /** Default limit on concurrent client-side zookeeper connections */ public static final int DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS = 300; @@ -244,21 +229,19 @@ public final class HConstants { public static final int DEFAULT_REGIONSERVER_INFOPORT = 16030; /** A configuration key for regionserver info port */ - public static final String REGIONSERVER_INFO_PORT = - "hbase.regionserver.info.port"; + public static final String REGIONSERVER_INFO_PORT = "hbase.regionserver.info.port"; /** A flag that enables automatic selection of regionserver info port */ - public static final String REGIONSERVER_INFO_PORT_AUTO = - REGIONSERVER_INFO_PORT + ".auto"; + public static final String REGIONSERVER_INFO_PORT_AUTO = REGIONSERVER_INFO_PORT + ".auto"; /** Parameter name for what region server implementation to use. */ - public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl"; + public static final String REGION_SERVER_IMPL = "hbase.regionserver.impl"; /** Parameter name for what master implementation to use. */ - public static final String MASTER_IMPL= "hbase.master.impl"; + public static final String MASTER_IMPL = "hbase.master.impl"; /** Parameter name for what hbase client implementation to use. */ - public static final String HBASECLIENT_IMPL= "hbase.hbaseclient.impl"; + public static final String HBASECLIENT_IMPL = "hbase.hbaseclient.impl"; /** Parameter name for how often threads should wake up */ public static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency"; @@ -293,7 +276,7 @@ public final class HConstants { /** Parameter name for HBase client operation timeout, which overrides RPC timeout */ public static final String HBASE_CLIENT_META_OPERATION_TIMEOUT = - "hbase.client.meta.operation.timeout"; + "hbase.client.meta.operation.timeout"; /** Default HBase client operation timeout, which is tantamount to a blocking call */ public static final int DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT = 1200000; @@ -316,9 +299,8 @@ public final class HConstants { public static final String MIGRATION_NAME = ".migration"; /** - * The directory from which co-processor/custom filter jars can be loaded - * dynamically by the region servers. This value can be overridden by the - * hbase.dynamic.jars.dir config. + * The directory from which co-processor/custom filter jars can be loaded dynamically by the + * region servers. This value can be overridden by the hbase.dynamic.jars.dir config. */ public static final String LIB_DIR = "lib"; @@ -326,8 +308,7 @@ public final class HConstants { public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir"; /** Conf key for the max file size after which we split the region */ - public static final String HREGION_MAX_FILESIZE = - "hbase.hregion.max.filesize"; + public static final String HREGION_MAX_FILESIZE = "hbase.hregion.max.filesize"; /** Default maximum file size */ public static final long DEFAULT_MAX_FILE_SIZE = 10 * 1024 * 1024 * 1024L; @@ -343,25 +324,24 @@ public final class HConstants { public static final long TABLE_MAX_ROWSIZE_DEFAULT = 1024 * 1024 * 1024L; /** - * The max number of threads used for opening and closing stores or store - * files in parallel + * The max number of threads used for opening and closing stores or store files in parallel */ public static final String HSTORE_OPEN_AND_CLOSE_THREADS_MAX = - "hbase.hstore.open.and.close.threads.max"; + "hbase.hstore.open.and.close.threads.max"; /** - * The default number for the max number of threads used for opening and - * closing stores or store files in parallel + * The default number for the max number of threads used for opening and closing stores or store + * files in parallel */ public static final int DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX = 1; /** - * Block updates if memstore has hbase.hregion.memstore.block.multiplier - * times hbase.hregion.memstore.flush.size bytes. Useful preventing - * runaway memstore during spikes in update traffic. + * Block updates if memstore has hbase.hregion.memstore.block.multiplier times + * hbase.hregion.memstore.flush.size bytes. Useful preventing runaway memstore during spikes in + * update traffic. */ public static final String HREGION_MEMSTORE_BLOCK_MULTIPLIER = - "hbase.hregion.memstore.block.multiplier"; + "hbase.hregion.memstore.block.multiplier"; /** * Default value for hbase.hregion.memstore.block.multiplier @@ -369,14 +349,12 @@ public final class HConstants { public static final int DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER = 4; /** Conf key for the memstore size at which we flush the memstore */ - public static final String HREGION_MEMSTORE_FLUSH_SIZE = - "hbase.hregion.memstore.flush.size"; + public static final String HREGION_MEMSTORE_FLUSH_SIZE = "hbase.hregion.memstore.flush.size"; public static final String HREGION_EDITS_REPLAY_SKIP_ERRORS = "hbase.hregion.edits.replay.skip.errors"; - public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS = - false; + public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS = false; /** Maximum value length, enforced on KeyValue construction */ public static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE - 1; @@ -412,12 +390,12 @@ public final class HConstants { // be the first to be reassigned if the server(s) they are being served by // should go down. - /** * The hbase:meta table's name. * @deprecated For upgrades of 0.94 to 0.96 */ - @Deprecated // for compat from 0.94 -> 0.96. + @Deprecated + // for compat from 0.94 -> 0.96. public static final byte[] META_TABLE_NAME = TableName.META_TABLE_NAME.getName(); public static final String BASE_NAMESPACE_DIR = "data"; @@ -425,52 +403,52 @@ public final class HConstants { /** delimiter used between portions of a region name */ public static final int META_ROW_DELIMITER = ','; - /** The catalog family as a string*/ + /** The catalog family as a string */ public static final String CATALOG_FAMILY_STR = "info"; /** The catalog family */ - public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR); + public static final byte[] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR); /** The RegionInfo qualifier as a string */ public static final String REGIONINFO_QUALIFIER_STR = "regioninfo"; /** The regioninfo column qualifier */ - public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR); + public static final byte[] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR); /** The server column qualifier */ public static final String SERVER_QUALIFIER_STR = "server"; /** The server column qualifier */ - public static final byte [] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR); + public static final byte[] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR); /** The startcode column qualifier */ public static final String STARTCODE_QUALIFIER_STR = "serverstartcode"; /** The startcode column qualifier */ - public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR); + public static final byte[] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR); /** The open seqnum column qualifier */ public static final String SEQNUM_QUALIFIER_STR = "seqnumDuringOpen"; /** The open seqnum column qualifier */ - public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR); + public static final byte[] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR); /** The state column qualifier */ public static final String STATE_QUALIFIER_STR = "state"; - public static final byte [] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR); + public static final byte[] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR); /** - * The serverName column qualifier. Its the server where the region is - * transitioning on, while column server is the server where the region is - * opened on. They are the same when the region is in state OPEN. + * The serverName column qualifier. Its the server where the region is transitioning on, while + * column server is the server where the region is opened on. They are the same when the region is + * in state OPEN. */ public static final String SERVERNAME_QUALIFIER_STR = "sn"; - public static final byte [] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR); + public static final byte[] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR); /** The lower-half split region column qualifier */ - public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA"); + public static final byte[] SPLITA_QUALIFIER = Bytes.toBytes("splitA"); /** The upper-half split region column qualifier */ - public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB"); + public static final byte[] SPLITB_QUALIFIER = Bytes.toBytes("splitB"); /** The lower-half merge region column qualifier */ public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes("mergeA"); @@ -478,32 +456,28 @@ public final class HConstants { /** The upper-half merge region column qualifier */ public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB"); - /** The catalog family as a string*/ + /** The catalog family as a string */ public static final String TABLE_FAMILY_STR = "table"; /** The catalog family */ - public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR); + public static final byte[] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR); /** The serialized table state qualifier */ public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state"); - /** - * The meta table version column qualifier. - * We keep current version of the meta table in this column in -ROOT- - * table: i.e. in the 'info:v' column. + * The meta table version column qualifier. We keep current version of the meta table in this + * column in -ROOT- table: i.e. in the 'info:v' column. */ - public static final byte [] META_VERSION_QUALIFIER = Bytes.toBytes("v"); + public static final byte[] META_VERSION_QUALIFIER = Bytes.toBytes("v"); /** - * The current version of the meta table. - * - pre-hbase 0.92. There is no META_VERSION column in the root table - * in this case. The meta has HTableDescriptor serialized into the HRegionInfo; - * - version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using - * Writable serialization, and HRegionInfo's does not contain HTableDescriptors. - * - version 1 for 0.96+ keeps HRegionInfo data structures, but changes the - * byte[] serialization from Writables to Protobuf. - * See HRegionInfo.VERSION + * The current version of the meta table. - pre-hbase 0.92. There is no META_VERSION column in the + * root table in this case. The meta has HTableDescriptor serialized into the HRegionInfo; - + * version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using Writable + * serialization, and HRegionInfo's does not contain HTableDescriptors. - version 1 for 0.96+ + * keeps HRegionInfo data structures, but changes the byte[] serialization from Writables to + * Protobuf. See HRegionInfo.VERSION */ public static final short META_VERSION = 1; @@ -512,25 +486,24 @@ public final class HConstants { /** * An empty instance. */ - public static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; + public static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(EMPTY_BYTE_ARRAY); /** * Used by scanners, etc when they want to start at the beginning of a region */ - public static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; + public static final byte[] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; /** * Last row in a table. */ - public static final byte [] EMPTY_END_ROW = EMPTY_START_ROW; + public static final byte[] EMPTY_END_ROW = EMPTY_START_ROW; /** - * Used by scanners and others when they're trying to detect the end of a - * table - */ - public static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY; + * Used by scanners and others when they're trying to detect the end of a table + */ + public static final byte[] LAST_ROW = EMPTY_BYTE_ARRAY; /** * Max length a row can have because of the limitation in TFile. @@ -538,9 +511,8 @@ public final class HConstants { public static final int MAX_ROW_LENGTH = Short.MAX_VALUE; /** - * Timestamp to use when we want to refer to the latest cell. - * This is the timestamp sent by clients when no timestamp is specified on - * commit. + * Timestamp to use when we want to refer to the latest cell. This is the timestamp sent by + * clients when no timestamp is specified on commit. */ public static final long LATEST_TIMESTAMP = Long.MAX_VALUE; @@ -552,17 +524,12 @@ public final class HConstants { /** * LATEST_TIMESTAMP in bytes form */ - public static final byte [] LATEST_TIMESTAMP_BYTES = { + public static final byte[] LATEST_TIMESTAMP_BYTES = { // big-endian - (byte) (LATEST_TIMESTAMP >>> 56), - (byte) (LATEST_TIMESTAMP >>> 48), - (byte) (LATEST_TIMESTAMP >>> 40), - (byte) (LATEST_TIMESTAMP >>> 32), - (byte) (LATEST_TIMESTAMP >>> 24), - (byte) (LATEST_TIMESTAMP >>> 16), - (byte) (LATEST_TIMESTAMP >>> 8), - (byte) LATEST_TIMESTAMP, - }; + (byte) (LATEST_TIMESTAMP >>> 56), (byte) (LATEST_TIMESTAMP >>> 48), + (byte) (LATEST_TIMESTAMP >>> 40), (byte) (LATEST_TIMESTAMP >>> 32), + (byte) (LATEST_TIMESTAMP >>> 24), (byte) (LATEST_TIMESTAMP >>> 16), + (byte) (LATEST_TIMESTAMP >>> 8), (byte) LATEST_TIMESTAMP }; /** * Define for 'return-all-versions'. @@ -572,7 +539,7 @@ public final class HConstants { /** * Unlimited time-to-live. */ -// public static final int FOREVER = -1; + // public static final int FOREVER = -1; public static final int FOREVER = Integer.MAX_VALUE; /** @@ -587,10 +554,10 @@ public final class HConstants { public static final int HOUR_IN_SECONDS = 60 * 60; public static final int MINUTE_IN_SECONDS = 60; - //TODO: although the following are referenced widely to format strings for - // the shell. They really aren't a part of the public API. It would be - // nice if we could put them somewhere where they did not need to be - // public. They could have package visibility + // TODO: although the following are referenced widely to format strings for + // the shell. They really aren't a part of the public API. It would be + // nice if we could put them somewhere where they did not need to be + // public. They could have package visibility public static final String NAME = "NAME"; public static final String VERSIONS = "VERSIONS"; public static final String IN_MEMORY = "IN_MEMORY"; @@ -598,45 +565,38 @@ public final class HConstants { public static final String CONFIGURATION = "CONFIGURATION"; /** - * Retrying we multiply hbase.client.pause setting by what we have in this array until we - * run out of array items. Retries beyond this use the last number in the array. So, for - * example, if hbase.client.pause is 1 second, and maximum retries count - * hbase.client.retries.number is 10, we will retry at the following intervals: - * 1, 2, 3, 5, 10, 20, 40, 100, 100, 100. - * With 100ms, a back-off of 200 means 20s + * Retrying we multiply hbase.client.pause setting by what we have in this array until we run out + * of array items. Retries beyond this use the last number in the array. So, for example, if + * hbase.client.pause is 1 second, and maximum retries count hbase.client.retries.number is 10, we + * will retry at the following intervals: 1, 2, 3, 5, 10, 20, 40, 100, 100, 100. With 100ms, a + * back-off of 200 means 20s */ - public static final int [] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200}; + public static final int[] RETRY_BACKOFF = + { 1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200 }; public static final String REGION_IMPL = "hbase.hregion.impl"; /** modifyTable op for replacing the table descriptor */ @InterfaceAudience.Private public static enum Modify { - CLOSE_REGION, - TABLE_COMPACT, - TABLE_FLUSH, - TABLE_MAJOR_COMPACT, - TABLE_SET_HTD, - TABLE_SPLIT + CLOSE_REGION, TABLE_COMPACT, TABLE_FLUSH, TABLE_MAJOR_COMPACT, TABLE_SET_HTD, TABLE_SPLIT } /** - * Scope tag for locally scoped data. - * This data will not be replicated. + * Scope tag for locally scoped data. This data will not be replicated. */ public static final int REPLICATION_SCOPE_LOCAL = 0; /** - * Scope tag for globally scoped data. - * This data will be replicated to all peers. + * Scope tag for globally scoped data. This data will be replicated to all peers. */ public static final int REPLICATION_SCOPE_GLOBAL = 1; /** - * Default cluster ID, cannot be used to identify a cluster so a key with - * this value means it wasn't meant for replication. + * Default cluster ID, cannot be used to identify a cluster so a key with this value means it + * wasn't meant for replication. */ - public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L,0L); + public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L, 0L); /** * Parameter name for maximum number of bytes returned when calling a scanner's next method. @@ -653,27 +613,22 @@ public final class HConstants { "hbase.server.scanner.max.result.size"; /** - * Maximum number of bytes returned when calling a scanner's next method. - * Note that when a single row is larger than this limit the row is still - * returned completely. - * - * The default value is 2MB. + * Maximum number of bytes returned when calling a scanner's next method. Note that when a single + * row is larger than this limit the row is still returned completely. The default value is 2MB. */ public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = 2 * 1024 * 1024; /** - * Maximum number of bytes returned when calling a scanner's next method. - * Note that when a single row is larger than this limit the row is still - * returned completely. - * Safety setting to protect the region server. - * - * The default value is 100MB. (a client would rarely request larger chunks on purpose) + * Maximum number of bytes returned when calling a scanner's next method. Note that when a single + * row is larger than this limit the row is still returned completely. Safety setting to protect + * the region server. The default value is 100MB. (a client would rarely request larger chunks on + * purpose) */ public static final long DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE = 100 * 1024 * 1024; /** - * Parameter name for client pause value, used mostly as value to wait - * before running a retry of a failed get, region lookup, etc. + * Parameter name for client pause value, used mostly as value to wait before running a retry of a + * failed get, region lookup, etc. */ public static final String HBASE_CLIENT_PAUSE = "hbase.client.pause"; @@ -693,8 +648,7 @@ public final class HConstants { public static final int DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS = 100; /** - * The maximum number of concurrent connections the client will maintain to a single - * RegionServer. + * The maximum number of concurrent connections the client will maintain to a single RegionServer. */ public static final String HBASE_CLIENT_MAX_PERSERVER_TASKS = "hbase.client.max.perserver.tasks"; @@ -704,8 +658,7 @@ public final class HConstants { public static final int DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS = 2; /** - * The maximum number of concurrent connections the client will maintain to a single - * Region. + * The maximum number of concurrent connections the client will maintain to a single Region. */ public static final String HBASE_CLIENT_MAX_PERREGION_TASKS = "hbase.client.max.perregion.tasks"; @@ -715,8 +668,8 @@ public final class HConstants { public static final int DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS = 1; /** - * Parameter name for server pause value, used mostly as value to wait before - * running a retry of a failed operation. + * Parameter name for server pause value, used mostly as value to wait before running a retry of a + * failed operation. */ public static final String HBASE_SERVER_PAUSE = "hbase.server.pause"; @@ -726,9 +679,9 @@ public final class HConstants { public static final int DEFAULT_HBASE_SERVER_PAUSE = 1000; /** - * Parameter name for maximum retries, used as maximum for all retryable - * operations such as fetching of the root region from root region server, - * getting a cell's value, starting a row update, etc. + * Parameter name for maximum retries, used as maximum for all retryable operations such as + * fetching of the root region from root region server, getting a cell's value, starting a row + * update, etc. */ public static final String HBASE_CLIENT_RETRIES_NUMBER = "hbase.client.retries.number"; @@ -748,10 +701,9 @@ public final class HConstants { public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = Integer.MAX_VALUE; /** - * Parameter name for number of rows that will be fetched when calling next on - * a scanner if it is not served from memory. Higher caching values will - * enable faster scanners but will eat up more memory and some calls of next - * may take longer and longer times when the cache is empty. + * Parameter name for number of rows that will be fetched when calling next on a scanner if it is + * not served from memory. Higher caching values will enable faster scanners but will eat up more + * memory and some calls of next may take longer and longer times when the cache is empty. */ public static final String HBASE_META_SCANNER_CACHING = "hbase.meta.scanner.caching"; @@ -918,18 +870,13 @@ public final class HConstants { public static final String LOCALHOST = "localhost"; /** - * If this parameter is set to true, then hbase will read - * data and then verify checksums. Checksum verification - * inside hdfs will be switched off. However, if the hbase-checksum - * verification fails, then it will switch back to using - * hdfs checksums for verifiying data that is being read from storage. - * - * If this parameter is set to false, then hbase will not - * verify any checksums, instead it will depend on checksum verification - * being done in the hdfs client. + * If this parameter is set to true, then hbase will read data and then verify checksums. Checksum + * verification inside hdfs will be switched off. However, if the hbase-checksum verification + * fails, then it will switch back to using hdfs checksums for verifiying data that is being read + * from storage. If this parameter is set to false, then hbase will not verify any checksums, + * instead it will depend on checksum verification being done in the hdfs client. */ - public static final String HBASE_CHECKSUM_VERIFICATION = - "hbase.regionserver.checksum.verify"; + public static final String HBASE_CHECKSUM_VERIFICATION = "hbase.regionserver.checksum.verify"; public static final String LOCALHOST_IP = "127.0.0.1"; @@ -944,17 +891,15 @@ public final class HConstants { public static final int DEFAULT_REGION_SERVER_HANDLER_COUNT = 30; /* - * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT: - * -1 => Disable aborting - * 0 => Abort if even a single handler has died - * 0.x => Abort only when this percent of handlers have died - * 1 => Abort only all of the handers have died + * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT: -1 => Disable aborting 0 => Abort if even a + * single handler has died 0.x => Abort only when this percent of handlers have died 1 => Abort + * only all of the handers have died */ public static final String REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT = "hbase.regionserver.handler.abort.on.error.percent"; public static final double DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT = 0.5; - //High priority handlers to deal with admin requests and system table operation requests + // High priority handlers to deal with admin requests and system table operation requests public static final String REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT = "hbase.regionserver.metahandler.count"; public static final int DEFAULT_REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT = 20; @@ -977,19 +922,16 @@ public final class HConstants { public static final int DEFAULT_META_REPLICA_NUM = 1; /** - * The name of the configuration parameter that specifies - * the number of bytes in a newly created checksum chunk. + * The name of the configuration parameter that specifies the number of bytes in a newly created + * checksum chunk. */ - public static final String BYTES_PER_CHECKSUM = - "hbase.hstore.bytes.per.checksum"; + public static final String BYTES_PER_CHECKSUM = "hbase.hstore.bytes.per.checksum"; /** - * The name of the configuration parameter that specifies - * the name of an algorithm that is used to compute checksums - * for newly created blocks. + * The name of the configuration parameter that specifies the name of an algorithm that is used to + * compute checksums for newly created blocks. */ - public static final String CHECKSUM_TYPE_NAME = - "hbase.hstore.checksum.algorithm"; + public static final String CHECKSUM_TYPE_NAME = "hbase.hstore.checksum.algorithm"; /** Enable file permission modification from standard hbase */ public static final String ENABLE_DATA_FILE_UMASK = "hbase.data.umask.enable"; @@ -997,16 +939,14 @@ public final class HConstants { public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask"; /** Configuration name of WAL Compression */ - public static final String ENABLE_WAL_COMPRESSION = - "hbase.regionserver.wal.enablecompression"; + public static final String ENABLE_WAL_COMPRESSION = "hbase.regionserver.wal.enablecompression"; - /** Configuration name of WAL storage policy - * Valid values are: - * NONE: no preference in destination of block replicas - * ONE_SSD: place only one block replica in SSD and the remaining in default storage - * and ALL_SSD: place all block replicas on SSD - * - * See http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html*/ + /** + * Configuration name of WAL storage policy Valid values are: NONE: no preference in destination + * of block replicas ONE_SSD: place only one block replica in SSD and the remaining in default + * storage and ALL_SSD: place all block replicas on SSD See + * http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html + */ public static final String WAL_STORAGE_POLICY = "hbase.wal.storage.policy"; public static final String DEFAULT_WAL_STORAGE_POLICY = "NONE"; @@ -1017,17 +957,16 @@ public final class HConstants { public static final String LOAD_BALANCER_SLOP_KEY = "hbase.regions.slop"; /** - * The byte array represents for NO_NEXT_INDEXED_KEY; - * The actual value is irrelevant because this is always compared by reference. + * The byte array represents for NO_NEXT_INDEXED_KEY; The actual value is irrelevant because this + * is always compared by reference. */ public static final Cell NO_NEXT_INDEXED_KEY = new KeyValue(); /** delimiter used between portions of a region name */ public static final int DELIMITER = ','; /** - * QOS attributes: these attributes are used to demarcate RPC call processing - * by different set of handlers. For example, HIGH_QOS tagged methods are - * handled by high priority handlers. + * QOS attributes: these attributes are used to demarcate RPC call processing by different set of + * handlers. For example, HIGH_QOS tagged methods are handled by high priority handlers. */ // normal_QOS < QOS_threshold < replication_QOS < replay_QOS < admin_QOS < high_QOS public static final int NORMAL_QOS = 0; @@ -1042,8 +981,8 @@ public final class HConstants { public static final String HFILE_ARCHIVE_DIRECTORY = "archive"; /** - * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for - * remaining snapshot constants; this is here to keep HConstants dependencies at a minimum and + * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for remaining + * snapshot constants; this is here to keep HConstants dependencies at a minimum and * uni-directional. */ public static final String SNAPSHOT_DIR_NAME = ".hbase-snapshot"; @@ -1059,49 +998,42 @@ public final class HConstants { public static final String REGIONSERVER_METRICS_PERIOD = "hbase.regionserver.metrics.period"; public static final long DEFAULT_REGIONSERVER_METRICS_PERIOD = 5000; /** Directories that are not HBase table directories */ - public static final List HBASE_NON_TABLE_DIRS = - Collections.unmodifiableList(Arrays.asList(new String[] { - HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME - })); + public static final List HBASE_NON_TABLE_DIRS = Collections.unmodifiableList(Arrays + .asList(new String[] { HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME })); /** Directories that are not HBase user table directories */ - public static final List HBASE_NON_USER_TABLE_DIRS = - Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll( - new String[] { TableName.META_TABLE_NAME.getNameAsString() }, - HBASE_NON_TABLE_DIRS.toArray()))); + public static final List HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays + .asList((String[]) ArrayUtils.addAll( + new String[] { TableName.META_TABLE_NAME.getNameAsString() }, + HBASE_NON_TABLE_DIRS.toArray()))); /** Health script related settings. */ public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location"; public static final String HEALTH_SCRIPT_TIMEOUT = "hbase.node.health.script.timeout"; - public static final String HEALTH_CHORE_WAKE_FREQ = - "hbase.node.health.script.frequency"; + public static final String HEALTH_CHORE_WAKE_FREQ = "hbase.node.health.script.frequency"; public static final long DEFAULT_HEALTH_SCRIPT_TIMEOUT = 60000; /** * The maximum number of health check failures a server can encounter consecutively. */ - public static final String HEALTH_FAILURE_THRESHOLD = - "hbase.node.health.failure.threshold"; + public static final String HEALTH_FAILURE_THRESHOLD = "hbase.node.health.failure.threshold"; public static final int DEFAULT_HEALTH_FAILURE_THRESHOLD = 3; - /** - * Setting to activate, or not, the publication of the status by the master. Default - * notification is by a multicast message. + * Setting to activate, or not, the publication of the status by the master. Default notification + * is by a multicast message. */ public static final String STATUS_PUBLISHED = "hbase.status.published"; public static final boolean STATUS_PUBLISHED_DEFAULT = false; /** - * IP to use for the multicast status messages between the master and the clients. - * The default address is chosen as one among others within the ones suitable for multicast - * messages. + * IP to use for the multicast status messages between the master and the clients. The default + * address is chosen as one among others within the ones suitable for multicast messages. */ public static final String STATUS_MULTICAST_ADDRESS = "hbase.status.multicast.address.ip"; public static final String DEFAULT_STATUS_MULTICAST_ADDRESS = "226.1.1.3"; /** - * The address to use for binding the local socket for receiving multicast. Defaults to - * 0.0.0.0. + * The address to use for binding the local socket for receiving multicast. Defaults to 0.0.0.0. * @see HBASE-9961 */ public static final String STATUS_MULTICAST_BIND_ADDRESS = @@ -1134,7 +1066,7 @@ public final class HConstants { /** Configuration key for the name of the alternate master key for the cluster, a string */ public static final String CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY = - "hbase.crypto.master.alternate.key.name"; + "hbase.crypto.master.alternate.key.name"; /** Configuration key for the algorithm to use when encrypting the WAL, a string */ public static final String CRYPTO_WAL_ALGORITHM_CONF_KEY = "hbase.crypto.wal.algorithm"; @@ -1166,7 +1098,7 @@ public final class HConstants { /** Config for pluggable consensus provider */ public static final String HBASE_COORDINATED_STATE_MANAGER_CLASS = - "hbase.coordinated.state.manager.class"; + "hbase.coordinated.state.manager.class"; /** Configuration key for SplitLog manager timeout */ public static final String HBASE_SPLITLOG_MANAGER_TIMEOUT = "hbase.splitlog.manager.timeout"; @@ -1180,17 +1112,18 @@ public final class HConstants { // hbase-common? /** - * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path - * to the file that will host the file-based cache. See BucketCache#getIOEngineFromName() for - * list of supported ioengine options. - *

Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache. + * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path to the + * file that will host the file-based cache. See BucketCache#getIOEngineFromName() for list of + * supported ioengine options. + *

+ * Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache. */ public static final String BUCKET_CACHE_IOENGINE_KEY = "hbase.bucketcache.ioengine"; /** * When using bucket cache, this is a float that EITHER represents a percentage of total heap - * memory size to give to the cache (if < 1.0) OR, it is the capacity in - * megabytes of the cache. + * memory size to give to the cache (if < 1.0) OR, it is the capacity in megabytes of the + * cache. */ public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size"; @@ -1203,26 +1136,25 @@ public final class HConstants { public static final String HBASE_CLIENT_FAST_FAIL_MODE_ENABLED = "hbase.client.fast.fail.mode.enabled"; - public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT = - false; + public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT = false; public static final String HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS = "hbase.client.fastfail.threshold"; - public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT = - 60000; + public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT = 60000; public static final String HBASE_CLIENT_FAST_FAIL_CLEANUP_MS_DURATION_MS = "hbase.client.fast.fail.cleanup.duration"; - public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT = - 600000; + public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT = 600000; public static final String HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL = "hbase.client.fast.fail.interceptor.impl"; - /** Config key for if the server should send backpressure and if the client should listen to - * that backpressure from the server */ + /** + * Config key for if the server should send backpressure and if the client should listen to that + * backpressure from the server + */ public static final String ENABLE_CLIENT_BACKPRESSURE = "hbase.client.backpressure.enabled"; public static final boolean DEFAULT_ENABLE_CLIENT_BACKPRESSURE = false; @@ -1234,11 +1166,11 @@ public final class HConstants { public static final float DEFAULT_HEAP_OCCUPANCY_HIGH_WATERMARK = 0.98f; /** - * The max number of threads used for splitting storefiles in parallel during - * the region split process. + * The max number of threads used for splitting storefiles in parallel during the region split + * process. */ public static final String REGION_SPLIT_THREADS_MAX = - "hbase.regionserver.region.split.threads.max"; + "hbase.regionserver.region.split.threads.max"; /** Canary config keys */ public static final String HBASE_CANARY_WRITE_DATA_TTL_KEY = "hbase.canary.write.data.ttl"; @@ -1263,6 +1195,15 @@ public final class HConstants { public static final String ZK_SERVER_KEYTAB_FILE = "hbase.zookeeper.server.keytab.file"; public static final String ZK_SERVER_KERBEROS_PRINCIPAL = "hbase.zookeeper.server.kerberos.principal"; + /** + * Backup/Restore constants + */ + + public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable"; + public final static boolean BACKUP_ENABLE_DEFAULT = true; + public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl"; + // Default TTL = 1 year + public final static int BACKUP_SYSTEM_TTL_DEFAULT = 365 * 24 * 3600; private HConstants() { // Can't be instantiated with this ctor. diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 26aad714b50..807b02135f6 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -393,6 +393,11 @@ hbase-resource-bundle ${project.version} true + + + org.apache.hadoop + hadoop-distcp + ${hadoop-two.version} commons-httpclient @@ -406,6 +411,11 @@ commons-collections commons-collections + + org.apache.hadoop + hadoop-distcp + ${hadoop-two.version} + org.apache.hbase hbase-hadoop-compat diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java new file mode 100644 index 00000000000..28cf6274fee --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java @@ -0,0 +1,338 @@ +/** + * 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.Set; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.zookeeper.KeeperException; + +/** + * Backup HBase tables locally or on a remote cluster Serve as client entry point for the following + * features: - Full Backup provide local and remote back/restore for a list of tables - Incremental + * backup to build on top of full backup as daily/weekly backup - Convert incremental backup WAL + * files into hfiles - Merge several backup images into one(like merge weekly into monthly) - Add + * and remove table to and from Backup image - Cancel a backup process - Full backup based on + * existing snapshot - Describe information of a backup image + */ + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class BackupClient { + private static final Log LOG = LogFactory.getLog(BackupClient.class); + private static Options opt; + private static Configuration conf = null; + + private BackupClient() { + throw new AssertionError("Instantiating utility class..."); + } + + protected static void init() throws IOException { + // define supported options + opt = new Options(); + + opt.addOption("all", false, "All tables"); + opt.addOption("debug", false, "Enable debug loggings"); + opt.addOption("t", true, "Table name"); + + // create configuration instance + conf = getConf(); + + // disable irrelevant loggers to avoid it mess up command output + disableUselessLoggers(); + + } + + + public static void main(String[] args) throws IOException { + init(); + parseAndRun(args); + System.exit(0); + } + + /** + * Set the configuration from a given one. + * @param newConf A new given configuration + */ + public synchronized static void setConf(Configuration newConf) { + conf = newConf; + BackupUtil.setConf(newConf); + } + + public static Configuration getConf() { + if (conf == null) { + conf = BackupUtil.getConf(); + } + return conf; + } + + private static void disableUselessLoggers() { + // disable zookeeper log to avoid it mess up command output + Logger zkLogger = Logger.getLogger("org.apache.zookeeper"); + LOG.debug("Zookeeper log level before set: " + zkLogger.getLevel()); + zkLogger.setLevel(Level.OFF); + LOG.debug("Zookeeper log level after set: " + zkLogger.getLevel()); + + // disable hbase zookeeper tool log to avoid it mess up command output + Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper"); + LOG.debug("HBase zookeeper log level before set: " + hbaseZkLogger.getLevel()); + hbaseZkLogger.setLevel(Level.OFF); + LOG.debug("HBase Zookeeper log level after set: " + hbaseZkLogger.getLevel()); + + // disable hbase client log to avoid it mess up command output + Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client"); + LOG.debug("HBase client log level before set: " + hbaseClientLogger.getLevel()); + hbaseClientLogger.setLevel(Level.OFF); + LOG.debug("HBase client log level after set: " + hbaseClientLogger.getLevel()); + } + + private static void parseAndRun(String[] args) throws IOException { + + String cmd = null; + String[] remainArgs = null; + if (args == null || args.length == 0) { + BackupCommands.createCommand(BackupRestoreConstants.BACKUP_COMMAND.HELP, null).execute(); + } else { + cmd = args[0]; + remainArgs = new String[args.length - 1]; + if (args.length > 1) { + System.arraycopy(args, 1, remainArgs, 0, args.length - 1); + } + } + CommandLine cmdline = null; + try { + cmdline = new PosixParser().parse(opt, remainArgs); + } catch (ParseException e) { + LOG.error("Could not parse command", e); + System.exit(-1); + } + + BACKUP_COMMAND type = BACKUP_COMMAND.HELP; + if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(cmd)) { + type = BACKUP_COMMAND.CREATE; + } else if (BACKUP_COMMAND.HELP.name().equalsIgnoreCase(cmd)) { + type = BACKUP_COMMAND.HELP; + } else { + System.out.println("Unsupported command for backup: " + cmd); + } + + // enable debug logging + Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); + if (cmdline.hasOption("debug")) { + backupClientLogger.setLevel(Level.DEBUG); + } else { + backupClientLogger.setLevel(Level.INFO); + } + + BackupCommands.createCommand(type, cmdline).execute(); + } + + /** + * Send backup request to server, and monitor the progress if necessary + * @param backupType : full or incremental + * @param backupRootPath : the rooPath specified by user + * @param tableListStr : the table list specified by user + * @param snapshot : using existing snapshot if specified by user (in future jira) + * @return backupId backup id + * @throws IOException exception + * @throws KeeperException excpetion + */ + public static String create(String backupType, String backupRootPath, String tableListStr, + String snapshot) throws IOException { + + String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime(); + + // check target path first, confirm it doesn't exist before backup + boolean isTargetExist = false; + try { + isTargetExist = HBackupFileSystem.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: " + + newMsg; + LOG.error(newMsg); + throw new IOException(newMsg); + } else { + throw e; + } + } catch (RuntimeException e) { + LOG.error(e.getMessage()); + throw e; + } + + if (isTargetExist) { + LOG.info("Using existing backup root dir: " + backupRootPath); + } else { + LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created."); + } + + // table list specified for backup, trigger backup on specified tables + String tableList = tableListStr; + // (tableListStr == null) ? null : tableListStr.replaceAll( + // BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND, + // BackupRestoreConstants.TABLENAME_DELIMITER_IN_ZNODE); + try { + requestBackup(backupId, backupType, tableList, backupRootPath, snapshot); + } catch (RuntimeException e) { + String errMsg = e.getMessage(); + if (errMsg != null + && (errMsg.startsWith("Non-existing tables found") || errMsg + .startsWith("Snapshot is not found"))) { + LOG.error(errMsg + ", please check your command"); + throw e; + } else { + throw e; + } + } + return backupId; + } + + /** + * Prepare and submit Backup request + * @param backupId : backup_timestame (something like backup_1398729212626) + * @param backupType : full or incremental + * @param tableList : tables to be backuped + * @param targetRootDir : specified by user + * @param snapshot : use existing snapshot if specified by user (for future jira) + * @throws IOException exception + */ + protected static void requestBackup(String backupId, String backupType, String tableList, + String targetRootDir, String snapshot) throws IOException { + + Configuration conf = getConf(); + BackupManager backupManager = null; + BackupContext backupContext = null; + if (snapshot != null) { + LOG.warn("Snapshot option specified, backup type and table option will be ignored,\n" + + "full backup will be taken based on the given snapshot."); + throw new IOException("backup using existing Snapshot will be implemented in future jira"); + } + + HBaseAdmin hbadmin = null; + Connection conn = null; + try { + + backupManager = new BackupManager(conf); + String tables = tableList; + if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + Set incrTableSet = backupManager.getIncrementalBackupTableSet(); + if (incrTableSet.isEmpty()) { + LOG.warn("Incremental backup table set contains no table.\n" + + "Use 'backup create full' or 'backup stop' to \n " + + "change the tables covered by incremental backup."); + throw new RuntimeException("No table covered by incremental backup."); + } + StringBuilder sb = new StringBuilder(); + for (String tableName : incrTableSet) { + sb.append(tableName + " "); + } + LOG.info("Incremental backup for the following table set: " + sb.toString()); + tables = + sb.toString().trim() + .replaceAll(" ", BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + + // check whether table exists first before starting real request + if (tables != null) { + String[] tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + ArrayList noneExistingTableList = null; + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + for (String tableName : tableNames) { + if (!hbadmin.tableExists(TableName.valueOf(tableName))) { + if (noneExistingTableList == null) { + noneExistingTableList = new ArrayList(); + } + noneExistingTableList.add(tableName); + } + } + if (noneExistingTableList != null) { + if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + LOG.warn("Incremental backup table set contains no-exising table: " + + noneExistingTableList); + } else { + // Throw exception only in full mode - we try to backup non-existing table + throw new RuntimeException("Non-existing tables found in the table list: " + + noneExistingTableList); + } + } + } + + // if any target table backup dir already exist, then no backup action taken + String[] tableNames = null; + if (tables != null && !tables.equals("")) { + tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + if (tableNames != null && tableNames.length > 0) { + for (String table : tableNames) { + String targetTableBackupDir = + HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); + Path targetTableBackupDirPath = new Path(targetTableBackupDir); + FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf); + if (outputFs.exists(targetTableBackupDirPath)) { + throw new IOException("Target backup directory " + targetTableBackupDir + + " exists already."); + } + } + } + backupContext = + backupManager.createBackupContext(backupId, backupType, tables, targetRootDir, snapshot); + backupManager.initialize(); + backupManager.dispatchRequest(backupContext); + } catch (BackupException e) { + // suppress the backup exception wrapped within #initialize or #dispatchRequest, backup + // exception has already been handled normally + StackTraceElement[] stes = e.getStackTrace(); + for (StackTraceElement ste : stes) { + LOG.info(ste); + } + LOG.error("Backup Exception " + e.getMessage()); + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if (conn != null) { + conn.close(); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java new file mode 100644 index 00000000000..c7aff151a5b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java @@ -0,0 +1,147 @@ +/** + * 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.commons.cli.CommandLine; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * General backup commands, options and usage messages + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +final class BackupCommands { + + private static final String USAGE = "Usage: hbase backup COMMAND\n" + + "where COMMAND is one of:\n" + " create create a new backup image\n" + + "Enter \'help COMMAND\' to see help message for each command\n"; + + private static final String CREATE_CMD_USAGE = + "Usage: hbase backup create [tables] [-s name] [-convert] " + + "[-silent]\n" + " type \"full\" to create a full backup image;\n" + + " \"incremental\" to create an incremental backup image\n" + + " backup_root_path The full root path to store the backup image,\n" + + " the prefix can be gpfs, hdfs or webhdfs\n" + " Options:\n" + + " tables If no tables (\"\") are specified, all tables are backed up. " + + "Otherwise it is a\n" + " comma separated list of tables.\n" + + " -s name Use the specified snapshot for full backup\n" + + " -convert For an incremental backup, convert WAL files to HFiles\n"; + + interface Command { + void execute() throws IOException; + } + + private BackupCommands() { + throw new AssertionError("Instantiating utility class..."); + } + + static Command createCommand(BACKUP_COMMAND type, CommandLine cmdline) { + Command cmd = null; + switch (type) { + case CREATE: + cmd = new CreateCommand(cmdline); + break; + case HELP: + default: + cmd = new HelpCommand(cmdline); + break; + } + return cmd; + } + + private static class CreateCommand implements Command { + CommandLine cmdline; + + CreateCommand(CommandLine cmdline) { + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null || cmdline.getArgs() == null) { + System.out.println("ERROR: missing arguments"); + System.out.println(CREATE_CMD_USAGE); + System.exit(-1); + } + String[] args = cmdline.getArgs(); + if (args.length < 2 || args.length > 3) { + System.out.println("ERROR: wrong number of arguments"); + System.out.println(CREATE_CMD_USAGE); + System.exit(-1); + } + + if (!BackupRestoreConstants.BACKUP_TYPE_FULL.equalsIgnoreCase(args[0]) + && !BackupRestoreConstants.BACKUP_TYPE_INCR.equalsIgnoreCase(args[0])) { + System.out.println("ERROR: invalid backup type"); + System.out.println(CREATE_CMD_USAGE); + System.exit(-1); + } + + String snapshot = cmdline.hasOption('s') ? cmdline.getOptionValue('s') : null; + String tables = (args.length == 3) ? args[2] : null; + + try { + BackupClient.create(args[0], args[1], tables, snapshot); + } catch (RuntimeException e) { + System.out.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + } + + private static class HelpCommand implements Command { + CommandLine cmdline; + + HelpCommand(CommandLine cmdline) { + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + if (cmdline == null) { + System.out.println(USAGE); + System.exit(0); + } + + String[] args = cmdline.getArgs(); + if (args == null || args.length == 0) { + System.out.println(USAGE); + System.exit(0); + } + + if (args.length != 1) { + System.out.println("Only support check help message of a single command type"); + System.out.println(USAGE); + System.exit(0); + } + + String type = args[0]; + + if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(type)) { + System.out.println(CREATE_CMD_USAGE); + } // other commands will be supported in future jira + System.exit(0); + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java new file mode 100644 index 00000000000..e8d6444e5e3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java @@ -0,0 +1,318 @@ +/** + * 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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * An object to encapsulate the information for each backup request + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupContext implements Serializable { + + private static final long serialVersionUID = 2401435114454300992L; + + // backup id: a timestamp when we request the backup + private String backupId; + + // backup type, full or incremental + private String type; + + // target root directory for storing the backup files + private String targetRootDir; + + // overall backup status + private BackupHandler.BACKUPSTATUS flag; + + // overall backup phase + private BackupHandler.BACKUPPHASE phase; + + // overall backup failure message + private String failedMsg; + + // backup status map for all tables + private Map backupStatusMap; + + // actual start timestamp of the backup process + private long startTs; + + // actual end timestamp of the backup process, could be fail or complete + private long endTs; + + // the total bytes of incremental logs copied + private long totalBytesCopied; + + // for incremental backup, the location of the backed-up hlogs + private String hlogTargetDir = null; + + // incremental backup file list + transient private List incrBackupFileList; + + // new region server log timestamps for table set after distributed log roll + // key - table name, value - map of RegionServer hostname -> last log rolled timestamp + transient private HashMap> tableSetTimestampMap; + + // cancel flag + private boolean cancelled = false; + // backup progress string + + private String progress; + + public BackupContext() { + } + + public BackupContext(String backupId, String type, String[] tables, String targetRootDir, + String snapshot) { + super(); + + if (backupStatusMap == null) { + backupStatusMap = new HashMap(); + } + + this.backupId = backupId; + this.type = type; + this.targetRootDir = targetRootDir; + + this.addTables(tables); + + if (type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, backupId)); + } + + this.startTs = 0; + this.endTs = 0; + + } + + /** + * Set progress string + * @param msg progress message + */ + + public void setProgress(String msg) { + this.progress = msg; + } + + /** + * Get current progress msg + */ + public String getProgress() { + return progress; + } + + /** + * Mark cancel flag. + */ + public void markCancel() { + this.cancelled = true; + } + + /** + * Has been marked as cancelled or not. + * @return True if marked as cancelled + */ + public boolean isCancelled() { + return this.cancelled; + } + + public String getBackupId() { + return backupId; + } + + public void setBackupId(String backupId) { + this.backupId = backupId; + } + + public BackupStatus getBackupStatus(String table) { + return this.backupStatusMap.get(table); + } + + public String getFailedMsg() { + return failedMsg; + } + + public void setFailedMsg(String failedMsg) { + this.failedMsg = failedMsg; + } + + public long getStartTs() { + return startTs; + } + + public void setStartTs(long startTs) { + this.startTs = startTs; + } + + public long getEndTs() { + return endTs; + } + + public void setEndTs(long endTs) { + this.endTs = endTs; + } + + public long getTotalBytesCopied() { + return totalBytesCopied; + } + + public BackupHandler.BACKUPSTATUS getFlag() { + return flag; + } + + public void setFlag(BackupHandler.BACKUPSTATUS flag) { + this.flag = flag; + } + + public BackupHandler.BACKUPPHASE getPhase() { + return phase; + } + + public void setPhase(BackupHandler.BACKUPPHASE phase) { + this.phase = phase; + } + + public String getType() { + return type; + } + + public void setSnapshotName(String table, String snapshotName) { + this.backupStatusMap.get(table).setSnapshotName(snapshotName); + } + + public String getSnapshotName(String table) { + return this.backupStatusMap.get(table).getSnapshotName(); + } + + public List getSnapshotNames() { + List snapshotNames = new ArrayList(); + for (BackupStatus backupStatus : this.backupStatusMap.values()) { + snapshotNames.add(backupStatus.getSnapshotName()); + } + return snapshotNames; + } + + public Set getTables() { + return this.backupStatusMap.keySet(); + } + + public String getTableListAsString() { + return BackupUtil.concat(backupStatusMap.keySet(), ";"); + } + + public void addTables(String[] tables) { + for (String table : tables) { + BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId); + this.backupStatusMap.put(table, backupStatus); + } + } + + public String getTargetRootDir() { + return targetRootDir; + } + + public void setHlogTargetDir(String hlogTagetDir) { + this.hlogTargetDir = hlogTagetDir; + } + + public String getHLogTargetDir() { + return hlogTargetDir; + } + + public List getIncrBackupFileList() { + return incrBackupFileList; + } + + public List setIncrBackupFileList(List incrBackupFileList) { + this.incrBackupFileList = incrBackupFileList; + return this.incrBackupFileList; + } + + /** + * Set the new region server log timestamps after distributed log roll + * @param newTableSetTimestampMap table timestamp map + */ + public void setIncrTimestampMap(HashMap> newTableSetTimestampMap) { + this.tableSetTimestampMap = newTableSetTimestampMap; + } + + /** + * Get new region server log timestamps after distributed log roll + * @return new region server log timestamps + */ + public HashMap> getIncrTimestampMap() { + return this.tableSetTimestampMap; + } + + /** + * Get existing snapshot if backing up from existing snapshot. + * @return The existing snapshot, null if not backing up from existing snapshot + */ + public String getExistingSnapshot() { + // this feature will be supported in another Jira + return null; + } + + /** + * Check whether this backup context are for backing up from existing snapshot or not. + * @return true if it is for backing up from existing snapshot, otherwise false + */ + public boolean fromExistingSnapshot() { + // this feature will be supported in later jiras + return false; + } + + public String getTableBySnapshot(String snapshotName) { + for (Entry entry : this.backupStatusMap.entrySet()) { + if (snapshotName.equals(entry.getValue().getSnapshotName())) { + return entry.getKey(); + } + } + return null; + } + + public byte[] toByteArray() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(this); + return baos.toByteArray(); + } + + public static BackupContext fromByteArray(byte[] data) + throws IOException, ClassNotFoundException { + ByteArrayInputStream bais = new ByteArrayInputStream(data); + ObjectInputStream ois = new ObjectInputStream(bais); + return (BackupContext) ois.readObject(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java new file mode 100644 index 00000000000..223746da065 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.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.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface BackupCopyService extends Configurable { + static enum Type { + FULL, INCREMENTAL + } + + public int copy(BackupHandler backupHandler, Configuration conf, BackupCopyService.Type copyType, + String[] options) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java new file mode 100644 index 00000000000..fe0d0e21562 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java @@ -0,0 +1,85 @@ +/** + * 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.HBaseIOException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * Backup exception + */ +@SuppressWarnings("serial") +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupException extends HBaseIOException { + private BackupContext description; + + /** + * Some exception happened for a backup and don't even know the backup that it was about + * @param msg Full description of the failure + */ + public BackupException(String msg) { + super(msg); + } + + /** + * Some exception happened for a backup with a cause + * @param cause the cause + */ + public BackupException(Throwable cause) { + super(cause); + } + + /** + * Exception for the given backup that has no previous root cause + * @param msg reason why the backup failed + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, BackupContext desc) { + super(msg); + this.description = desc; + } + + /** + * Exception for the given backup due to another exception + * @param msg reason why the backup failed + * @param cause root cause of the failure + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, Throwable cause, BackupContext desc) { + super(msg, cause); + this.description = desc; + } + + /** + * Exception when the description of the backup cannot be determined, due to some other root + * cause + * @param message description of what caused the failure + * @param e root cause + */ + public BackupException(String message, Exception e) { + super(message, e); + } + + public BackupContext getBackupContext() { + return this.description; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java new file mode 100644 index 00000000000..f764f182a13 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java @@ -0,0 +1,744 @@ +/** + * 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.List; +import java.util.concurrent.Callable; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.zookeeper.KeeperException.NoNodeException; + +/** + * A Handler to carry the operations of backup progress + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupHandler implements Callable { + private static final Log LOG = LogFactory.getLog(BackupHandler.class); + + // backup phase + // for overall backup (for table list, some table may go online, while some may go offline) + protected static enum BACKUPPHASE { + REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST; + } + + // backup status flag + protected static enum BACKUPSTATUS { + WAITING, ONGOING, COMPLETE, FAILED, CANCELLED; + } + + protected BackupContext backupContext; + private BackupManager backupManager; + private Configuration conf; + + public BackupHandler(BackupContext backupContext, + BackupManager backupManager, Configuration conf) { + this.backupContext = backupContext; + this.backupManager = backupManager; + this.conf = conf; + } + + public BackupContext getBackupContext() { + return backupContext; + } + + @Override + public Object call() throws Exception { + try { + // overall backup begin + this.beginBackup(backupContext); + HashMap newTimestamps = null; + boolean fromExistingSnapshot = false; // supported by future jira + // handle full or incremental backup for table or table list + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) { + String savedStartCode = null; + HBaseAdmin hbadmin = null; + Connection conn = null; + boolean firstBackup = false; + // do snapshot for full table backup, if backing up from existing snapshot, then skip the + // step of taking snapshot + if (fromExistingSnapshot) { + LOG.error("Backup from existing snapshot, so skip the snapshot step. "); + LOG.error("This feature will be supported by a future jira"); + throw new RuntimeException("Backup from existing snapshot is not supported"); + } else { + try { + savedStartCode = backupManager.readBackupStartCode(); + firstBackup = savedStartCode == null; + if (firstBackup) { + // This is our first backup. Let's put some marker on ZK so that we can hold the logs + // while we do the backup. + backupManager.writeBackupStartCode("0"); + } + // 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 ..."); + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap()); + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + if (firstBackup) { + // Updates registered log files + // We record ALL old WAL files as registered, because + // this is a first full backup in the system and these + // files are not needed for next incremental backup + List logFiles = BackupUtil.getWALFilesOlderThan(conf, newTimestamps); + backupManager.recordWALFiles(logFiles); + } + this.snapshotForFullBackup(backupContext); + } catch (BackupException e) { + // fail the overall backup and return + this.failBackup(backupContext, e, "Unexpected BackupException : "); + return null; + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if (conn != null) { + conn.close(); + } + } + } + + // update the faked progress currently for snapshot done + this.updateProgress("10.0%", 0); + // do snapshot copy + try { + this.snapshotCopy(backupContext); + } catch (Exception e) { + // fail the overall backup and return + this.failBackup(backupContext, e, "Unexpected BackupException : "); + return null; + } + // Updates incremental backup table set + backupManager.addIncrementalBackupTableSet(backupContext.getTables()); + + } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + LOG.info("For incremental backup, current table set is " + + backupManager.getIncrementalBackupTableSet()); + // do incremental table backup preparation + backupContext.setPhase(BACKUPPHASE.PREPARE_INCREMENTAL); + // avoid action if has been cancelled + if (backupContext.isCancelled()) { + return null; + } + try { + IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager); + + newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext); + } catch (Exception e) { + // fail the overall backup and return + this.failBackup(backupContext, e, "Unexpected Exception : "); + return null; + } + // update the faked progress currently for incremental preparation done + this.updateProgress("10.0%", 0); + + // do incremental copy + try { + // copy out the table and region info files for each table + BackupUtil.copyTableRegionInfo(backupContext, conf); + this.incrementalCopy(backupContext); + // Save list of WAL files copied + backupManager.recordWALFiles(backupContext.getIncrBackupFileList()); + } catch (Exception e) { + // fail the overall backup and return + this.failBackup(backupContext, e, "Unexpected exception doing incremental copy : "); + return null; + } + + } else { + LOG.error("Unsupport backup type: " + backupContext.getType()); + } + + // set overall backup status: complete. Here we make sure to complete the backup. After this + // checkpoint, even if entering cancel process, will let the backup finished + backupContext.setFlag(BACKUPSTATUS.COMPLETE); + + if (!fromExistingSnapshot) { + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + // Set the previousTimestampMap which is before this current log roll to the manifest. + HashMap> previousTimestampMap = + backupManager.readLogTimestampMap(); + backupContext.setIncrTimestampMap(previousTimestampMap); + } + // The table list in backupContext is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + + backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps); + + HashMap> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + String newStartCode = + BackupUtil.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + } + + // backup complete + this.completeBackup(backupContext); + } catch (Exception e) { + // even during completing backup (#completeBackup(backupContext)), exception may occur, or + // exception occur during other process, fail the backup finally + this.failBackup(backupContext, e, "Error caught during backup progress: "); + } + return null; + } + + /** + * Begin the overall backup. + * @param backupContext backup context + * @throws IOException exception + */ + private void beginBackup(BackupContext backupContext) throws IOException { + + // set the start timestamp of the overall backup + long startTs = EnvironmentEdgeManager.currentTime(); + backupContext.setStartTs(startTs); + // set overall backup status: ongoing + backupContext.setFlag(BACKUPSTATUS.ONGOING); + LOG.info("Backup " + backupContext.getBackupId() + " starts at " + startTs + "."); + + backupManager.updateBackupStatus(backupContext); + LOG.debug("Backup session " + backupContext.getBackupId() + " has been started."); + + } + + /** + * Snapshot for full table backup. + * @param backupContext backup context + * @throws IOException exception + */ + private void snapshotForFullBackup(BackupContext backupContext) throws IOException { + + LOG.info("HBase snapshot full backup for " + backupContext.getBackupId()); + + // avoid action if has been cancelled + if (backupContext.isCancelled()) { + return; + } + + HBaseAdmin hbadmin = null; + Connection conn = null; + + // we do HBase snapshot for tables in the table list one by one currently + for (String table : backupContext.getTables()) { + + // avoid action if it has been cancelled + if (backupContext.isCancelled()) { + return; + } + + HBaseProtos.SnapshotDescription backupSnapshot; + try { + // wrap a SnapshotDescription for offline/online snapshot + backupSnapshot = this.wrapSnapshotDescription(table); + + // set the snapshot name in BackupStatus of this table + backupContext.setSnapshotName(table, backupSnapshot.getName()); + + // Kick off snapshot for backup + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + hbadmin.snapshot(backupSnapshot); + + if (LOG.isDebugEnabled() == false) { + // In DEBUG mode we log message already. + // This is not to duplicate that message. + LOG.info("Snapshot has been launched, waiting to finish ..."); + } + + } catch (Exception e) { + LOG.error("Snapshot failed to create " + getMessage(e)); + + // currently, we fail the overall backup if any table in the list failed, so throw the + // exception out for overall backup failing + throw new BackupException("Backup snapshot failed on table " + table, e); + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if (conn != null) { + conn.close(); + } + } + + // set the snapshot name in BackupStatus of this table, only after snapshot success. + backupContext.setSnapshotName(table, backupSnapshot.getName()); + + } // for each table in the backup table list + + } + + /** + * Fail the overall backup. + * @param backupContext backup context + * @param e exception + * @throws Exception exception + */ + private void failBackup(BackupContext backupContext, Exception e, String msg) throws Exception { + + LOG.error(msg + getMessage(e)); + + // If this is a cancel exception, then we've already cleaned. + + if (this.backupContext.getFlag().equals(BACKUPSTATUS.CANCELLED)) { + return; + } + + // set the failure timestamp of the overall backup + backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); + + // set failure message + backupContext.setFailedMsg(e.getMessage()); + + // set overall backup status: failed + backupContext.setFlag(BACKUPSTATUS.FAILED); + + // compose the backup failed data + String backupFailedData = + "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs() + + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase() + + ",failedmessage=" + backupContext.getFailedMsg(); + LOG.error(backupFailedData); + + backupManager.updateBackupStatus(backupContext); + + // if full backup, then delete HBase snapshots if there already have snapshots taken + // and also clean up export snapshot log files if exist + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) { + if (!backupContext.fromExistingSnapshot()) { + this.deleteSnapshot(backupContext); + } + this.cleanupExportSnapshotLog(); + } /* + * else { // support incremental backup code in future jira // TODO. See HBASE-14124 } + */ + + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + // For incremental backup, DistCp logs will be cleaned with the targetDir. + this.cleanupTargetDir(); + + LOG.info("Backup " + backupContext.getBackupId() + " failed."); + } + + /** + * Update the ongoing back token znode with new progress. + * @param newProgress progress + * @param bytesCopied bytes copied + * @throws NoNodeException exception + */ + public void updateProgress(String newProgress, long bytesCopied) throws IOException { + + // compose the new backup progress data, using fake number for now + String backupProgressData = newProgress; + + backupContext.setProgress(newProgress); + backupManager.updateBackupStatus(backupContext); + LOG.debug("Backup progress data \"" + backupProgressData + + "\" has been updated to hbase:backup for " + backupContext.getBackupId()); + } + + /** + * Complete the overall backup. + * @param backupContext backup context + * @throws Exception exception + */ + private void completeBackup(BackupContext backupContext) throws Exception { + + // set the complete timestamp of the overall backup + backupContext.setEndTs(EnvironmentEdgeManager.currentTime()); + // set overall backup status: complete + backupContext.setFlag(BACKUPSTATUS.COMPLETE); + // add and store the manifest for the backup + this.addManifest(backupContext); + + // after major steps done and manifest persisted, do convert if needed for incremental backup + /* in-fly convert code here, provided by future jira */ + LOG.debug("in-fly convert code here, provided by future jira"); + + // compose the backup complete data + String backupCompleteData = + this.obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs() + + ",completets=" + backupContext.getEndTs() + ",bytescopied=" + + backupContext.getTotalBytesCopied(); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData); + } + backupManager.updateBackupStatus(backupContext); + + // when full backup is done: + // - delete HBase snapshot + // - clean up directories with prefix "exportSnapshot-", which are generated when exporting + // snapshots + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) { + if (!backupContext.fromExistingSnapshot()) { + this.deleteSnapshot(backupContext); + } + this.cleanupExportSnapshotLog(); + } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + this.cleanupDistCpLog(); + } else { + LOG.error(" other backup types have not been implemented yet"); + } + + LOG.info("Backup " + backupContext.getBackupId() + " completed."); + } + + /** + * Get backup request meta data dir as string. + * @param backupContext backup context + * @return meta data dir + */ + private String obtainBackupMetaDataStr(BackupContext backupContext) { + StringBuffer sb = new StringBuffer(); + sb.append("type=" + backupContext.getType() + ",tablelist="); + for (String table : backupContext.getTables()) { + sb.append(table + ";"); + } + if (sb.lastIndexOf(";") > 0) { + sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); + } + sb.append(",targetRootDir=" + backupContext.getTargetRootDir()); + if (backupContext.fromExistingSnapshot()) { + sb.append(",snapshot=" + backupContext.getExistingSnapshot()); + } + return sb.toString(); + } + + /** + * Do snapshot copy. + * @param backupContext backup context + * @throws Exception exception + */ + private void snapshotCopy(BackupContext backupContext) throws Exception { + + LOG.info("Snapshot copy is starting."); + + // set overall backup phase: snapshot_copy + backupContext.setPhase(BACKUPPHASE.SNAPSHOTCOPY); + + // avoid action if has been cancelled + if (backupContext.isCancelled()) { + return; + } + + // call ExportSnapshot to copy files based on hbase snapshot for backup + // ExportSnapshot only support single snapshot export, need loop for multiple tables case + BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf); + + // number of snapshots matches number of tables + float numOfSnapshots = backupContext.getSnapshotNames().size(); + + LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied."); + + for (String table : backupContext.getTables()) { + // Currently we simply set the sub copy tasks by counting the table snapshot number, we can + // calculate the real files' size for the percentage in the future. + // TODO this below + // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots); + int res = 0; + String[] args = new String[4]; + args[0] = "-snapshot"; + args[1] = backupContext.getSnapshotName(table); + args[2] = "-copy-to"; + args[3] = backupContext.getBackupStatus(table).getTargetDir(); + + LOG.debug("Copy snapshot " + args[1] + " to " + args[3]); + res = copyService.copy(this, conf, BackupCopyService.Type.FULL, args); + // if one snapshot export failed, do not continue for remained snapshots + if (res != 0) { + + LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + "."); + + throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3] + + " with reason code " + res); + } + + LOG.info("Snapshot copy " + args[1] + " finished."); + } + } + + /** + * Wrap a SnapshotDescription for a target table. + * @param table table + * @return a SnapshotDescription especially for backup. + */ + private SnapshotDescription wrapSnapshotDescription(String table) { + // Mock a SnapshotDescription from backupContext to call SnapshotManager function, + // Name it in the format "snapshot__" + HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder(); + builder.setTable(table); + TableName tableName = TableName.valueOf(table); + builder.setName("snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_" + + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString()); + HBaseProtos.SnapshotDescription backupSnapshot = builder.build(); + + LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName() + + " from backupContext to request snapshot for backup."); + + return backupSnapshot; + } + + /** + * Delete HBase snapshot for backup. + * @param backupCtx backup context + * @throws Exception exception + */ + private void deleteSnapshot(BackupContext backupCtx) throws IOException { + + LOG.debug("Trying to delete snapshot for full backup."); + Connection conn = null; + Admin admin = null; + try { + conn = ConnectionFactory.createConnection(conf); + admin = conn.getAdmin(); + for (String snapshotName : backupCtx.getSnapshotNames()) { + if (snapshotName == null) { + continue; + } + LOG.debug("Trying to delete snapshot: " + snapshotName); + admin.deleteSnapshot(snapshotName); + LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + + backupCtx.getBackupId() + " succeeded."); + } + } finally { + if (admin != null) { + admin.close(); + } + if (conn != null) { + conn.close(); + } + } + } + + /** + * Clean up directories with prefix "exportSnapshot-", which are generated when exporting + * snapshots. + * @throws IOException exception + */ + private void cleanupExportSnapshotLog() throws IOException { + FileSystem fs = FSUtils.getCurrentFileSystem(conf); + Path stagingDir = + new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory() + .toString())); + FileStatus[] files = FSUtils.listStatus(fs, stagingDir); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("exportSnapshot-")) { + LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); + if (FSUtils.delete(fs, file.getPath(), true) == false) { + LOG.warn("Can not delete " + file.getPath()); + } + } + } + } + + /** + * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying + * hlogs. + * @throws IOException exception + */ + private void cleanupDistCpLog() throws IOException { + Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = FSUtils.listStatus(fs, rootPath); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("_distcp_logs")) { + LOG.debug("Delete log files of DistCp: " + file.getPath().getName()); + FSUtils.delete(fs, file.getPath(), true); + } + } + } + + /** + * Clean up the uncompleted data at target directory if the ongoing backup has already entered the + * copy phase. + */ + private void cleanupTargetDir() { + try { + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + LOG.debug("Trying to cleanup up target dir. Current backup phase: " + + backupContext.getPhase()); + if (backupContext.getPhase().equals(BACKUPPHASE.SNAPSHOTCOPY) + || backupContext.getPhase().equals(BACKUPPHASE.INCREMENTAL_COPY) + || backupContext.getPhase().equals(BACKUPPHASE.STORE_MANIFEST)) { + FileSystem outputFs = + FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf); + + // now treat one backup as a transaction, clean up data that has been partially copied at + // table level + for (String table : backupContext.getTables()) { + Path targetDirPath = + new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(), + backupContext.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString() + + " done."); + } else { + LOG.info("No data has been copied to " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir); + if (backups == null || backups.length == 0) { + outputFs.delete(tableDir, true); + LOG.debug(tableDir.toString() + " is empty, remove it."); + } + } + } + + } catch (IOException e1) { + LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at " + + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Add manifest for the current backup. The manifest is stored + * within the table backup directory. + * @param backupContext The current backup context + * @throws IOException exception + * @throws BackupException exception + */ + private void addManifest(BackupContext backupContext) throws IOException, BackupException { + // set the overall backup phase : store manifest + backupContext.setPhase(BACKUPPHASE.STORE_MANIFEST); + + // avoid action if has been cancelled + if (backupContext.isCancelled()) { + return; + } + + BackupManifest manifest; + boolean fromExistingSnapshot = false; // to be implemented in future jira + + // Since we have each table's backup in its own directory structure, + // we'll store its manifest with the table directory. + for (String table : backupContext.getTables()) { + manifest = new BackupManifest(backupContext, table); + if (fromExistingSnapshot) { + // mark backing up from existing snapshot in manifest, so that later, dependency analysis + // can skip this backup image + LOG.debug("backup using existing snapshot will be supported in future jira"); + } else { + ArrayList ancestors = this.backupManager.getAncestors(backupContext, table); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + } + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + // We'll store the log timestamps for this table only in its manifest. + HashMap> tableTimestampMap = + new HashMap>(); + tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table)); + manifest.setIncrTimestampMap(tableTimestampMap); + } + manifest.store(conf); + } + + // For incremental backup, we store a overall manifest in + // /WALs/ + // This is used when created the next incremental backup + if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + manifest = new BackupManifest(backupContext); + // set the table region server start and end timestamps for incremental backup + manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap()); + ArrayList ancestors = this.backupManager.getAncestors(backupContext); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + manifest.store(conf); + } + } + + /** + * Do incremental copy. + * @param backupContext backup context + */ + private void incrementalCopy(BackupContext backupContext) throws Exception { + + LOG.info("Incremental copy is starting."); + + // set overall backup phase: incremental_copy + backupContext.setPhase(BACKUPPHASE.INCREMENTAL_COPY); + + // avoid action if has been cancelled + if (backupContext.isCancelled()) { + return; + } + + // get incremental backup file list and prepare parms for DistCp + List incrBackupFileList = backupContext.getIncrBackupFileList(); + String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]); + strArr[strArr.length - 1] = backupContext.getHLogTargetDir(); + + BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf); + int res = copyService.copy(this, conf, BackupCopyService.Type.INCREMENTAL, strArr); + + if (res != 0) { + LOG.error("Copy incremental log files failed with return code: " + res + "."); + throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to " + + backupContext.getHLogTargetDir()); + } + LOG.info("Incremental copy from " + incrBackupFileList + " to " + + backupContext.getHLogTargetDir() + " finished."); + + } + + private String getMessage(Exception e) { + String msg = e.getMessage(); + if (msg == null || msg.equals("")) { + msg = e.getClass().getName(); + } + return msg; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java new file mode 100644 index 00000000000..fdb3c466e00 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java @@ -0,0 +1,488 @@ +/** + * 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 com.google.common.util.concurrent.ThreadFactoryBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS; +import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.backup.master.BackupLogCleaner; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; + + + +/** + * Handles backup requests on server-side, creates backup context records in hbase:backup + * to keep track backup. The timestamps kept in hbase:backup table will be used for future + * incremental backup. Creates BackupContext and DispatchRequest. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupManager { + private static final Log LOG = LogFactory.getLog(BackupManager.class); + + private Configuration conf = null; + private BackupContext backupContext = null; + private ExecutorService pool = null; + + private boolean backupComplete = false; + + private BackupSystemTable systemTable; + + /** + * Backup manager constructor. + * @param conf configuration + * @throws IOException exception + */ + public BackupManager(Configuration conf) throws IOException { + if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) { + throw new BackupException("HBase backup is not enabled. Check your " + + HConstants.BACKUP_ENABLE_KEY + " setting."); + } + this.conf = conf; + this.systemTable = BackupSystemTable.getTable(conf); + Runtime.getRuntime().addShutdownHook(new ExitHandler()); + } + + /** + * This method modifies the master's configuration in order to inject backup-related features + * @param conf configuration + */ + public static void decorateMasterConfiguration(Configuration conf) { + if (!isBackupEnabled(conf)) { + return; + } + String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS); + String cleanerClass = BackupLogCleaner.class.getCanonicalName(); + if (!plugins.contains(cleanerClass)) { + conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Added log cleaner: " + cleanerClass); + } + } + + private static boolean isBackupEnabled(Configuration conf) { + return conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT); + } + + private class ExitHandler extends Thread { + public ExitHandler() { + super("Backup Manager Exit Handler"); + } + + public void run() { + + if (backupContext != null && !backupComplete) { + + // program exit and backup is not complete, then mark as cancelled to avoid submitted backup + // handler's taking further action + backupContext.markCancel(); + + LOG.debug("Backup is cancelled due to force program exiting."); + try { + cancelBackup(backupContext.getBackupId()); + } catch (Exception e) { + String msg = e.getMessage(); + if (msg == null || msg.equals("")) { + msg = e.getClass().getName(); + } + LOG.error("Failed to cancel backup " + backupContext.getBackupId() + " due to " + msg); + } + } + + exit(); + } + } + + /** + * Cancel the ongoing backup via backup id. + * @param backupId The id of the ongoing backup to be cancelled + * @throws Exception exception + */ + private void cancelBackup(String backupId) throws Exception { + // TODO: will be implemented in Phase 2: HBASE-14125 + LOG.debug("Try to cancel the backup " + backupId + ". the feature is NOT implemented yet"); + + } + + /** + * Stop all the work of backup. + */ + public void exit() { + + // currently, we shutdown now for all ongoing back handlers, we may need to do something like + // record the failed list somewhere later + if (this.pool != null) { + this.pool.shutdownNow(); + } + + } + + /** + * Create a BackupContext based on input backup request. + * @param backupId backup id + * @param type type + * @param tablelist table list + * @param targetRootDir root dir + * @param snapshot snapshot name + * @return BackupContext context + * @throws BackupException exception + */ + protected BackupContext createBackupContext(String backupId, String type, String tablelist, + String targetRootDir, String snapshot) throws BackupException { + + if (targetRootDir == null) { + throw new BackupException("Wrong backup request parameter: target backup root directory"); + } + + if (type.equals(BackupRestoreConstants.BACKUP_TYPE_FULL) && tablelist == null) { + // If table list is null for full backup, which means backup all tables. Then fill the table + // list with all user tables from meta. It no table available, throw the request exception. + + HTableDescriptor[] htds = null; + try (Connection conn = ConnectionFactory.createConnection(conf); + HBaseAdmin hbadmin = (HBaseAdmin) conn.getAdmin()) { + + htds = hbadmin.listTables(); + } catch (Exception e) { + throw new BackupException(e); + } + + if (htds == null) { + throw new BackupException("No table exists for full backup of all tables."); + } else { + StringBuilder sb = new StringBuilder(); + for (HTableDescriptor hTableDescriptor : htds) { + sb.append(hTableDescriptor.getNameAsString() + + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + sb.deleteCharAt(sb.lastIndexOf(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)); + tablelist = sb.toString(); + + LOG.info("Full backup all the tables available in the cluster: " + tablelist); + } + } + + // there are one or more tables in the table list + return new BackupContext(backupId, type, + tablelist.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND), targetRootDir, + snapshot); + + } + + /** + * Check if any ongoing backup. Currently, we only reply on checking status in hbase:backup. We + * need to consider to handle the case of orphan records in the future. Otherwise, all the coming + * request will fail. + * @return the ongoing backup id if on going backup exists, otherwise null + * @throws IOException exception + */ + private String getOngoingBackupId() throws IOException { + + ArrayList sessions = systemTable.getBackupContexts(BACKUPSTATUS.ONGOING); + if (sessions.size() == 0) { + return null; + } + return sessions.get(0).getBackupId(); + } + + /** + * Start the backup manager service. + * @throws IOException exception + */ + public void initialize() throws IOException { + String ongoingBackupId = this.getOngoingBackupId(); + if (ongoingBackupId != null) { + LOG.info("There is a ongoing backup " + ongoingBackupId + + ". Can not launch new backup until no ongoing backup remains."); + throw new BackupException("There is ongoing backup."); + } + + // Initialize thread pools + int nrThreads = this.conf.getInt("hbase.backup.threads.max", 1); + ThreadFactoryBuilder builder = new ThreadFactoryBuilder(); + builder.setNameFormat("BackupHandler-%1$d"); + this.pool = + new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS, + new LinkedBlockingQueue(), builder.build()); + ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); + } + + /** + * Dispatch and handle a backup request. + * @param backupContext backup context + * @throws BackupException exception + */ + public void dispatchRequest(BackupContext backupContext) throws BackupException { + + this.backupContext = backupContext; + + LOG.info("Got a backup request: " + "Type: " + backupContext.getType() + "; Tables: " + + backupContext.getTableListAsString() + "; TargetRootDir: " + + backupContext.getTargetRootDir()); + + // dispatch the request to a backup handler and put it handler map + + BackupHandler handler = new BackupHandler(this.backupContext, this, conf); + Future future = this.pool.submit(handler); + // wait for the execution to complete + try { + future.get(); + } catch (InterruptedException e) { + throw new BackupException(e); + } catch (CancellationException e) { + throw new BackupException(e); + } catch (ExecutionException e) { + throw new BackupException(e); + } + + // mark the backup complete for exit handler's processing + backupComplete = true; + + LOG.info("Backup request " + backupContext.getBackupId() + " has been executed."); + } + + /** + * Get direct ancestors of the current backup. + * @param backupCtx The backup context for the current backup + * @return The ancestors for the current backup + * @throws IOException exception + * @throws BackupException exception + */ + protected ArrayList getAncestors(BackupContext backupCtx) throws IOException, + BackupException { + LOG.debug("Getting the direct ancestors of the current backup ..."); + + ArrayList ancestors = new ArrayList(); + + // full backup does not have ancestor + if (backupCtx.getType().equals(BackupRestoreConstants.BACKUP_TYPE_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(); + for (BackupCompleteData backup : allHistoryList) { + BackupImage image = + new BackupImage(backup.getBackupToken(), backup.getType(), backup.getBackupRootPath(), + backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup + .getEndTime())); + + // add the full backup image as an ancestor until the last incremental backup + if (backup.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) { + + // backup image from existing snapshot does not involve in dependency + if (backup.fromExistingSnapshot()) { + continue; + } + // check the backup image coverage, if previous image could be covered by the newer ones, + // then no need to add + if (!BackupManifest.canCoverImage(ancestors, image)) { + ancestors.add(image); + } + } else { + // found last incremental backup, if previously added full backup ancestor images can cover + // it, then this incremental ancestor is not the dependent of the current incremental + // backup, that is to say, this is the backup scope boundary of current table set. + // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing + // incremental backup + if (BackupManifest.canCoverImage(ancestors, image)) { + LOG.debug("Met the backup boundary of the current table set. " + + "The root full backup images for the current backup scope:"); + for (BackupImage image1 : ancestors) { + LOG.debug(" BackupId: " + image1.getBackupId() + ", Backup directory: " + + image1.getRootDir()); + } + } else { + Path logBackupPath = + HBackupFileSystem.getLogBackupPath(backup.getBackupRootPath(), + backup.getBackupToken()); + LOG.debug("Current backup has an incremental backup ancestor, " + + "touching its image manifest in " + logBackupPath.toString() + + " to construct the dependency."); + + BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath); + BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage(); + ancestors.add(lastIncrImage); + + LOG.debug("Last dependent incremental backup image information:"); + LOG.debug(" Token: " + lastIncrImage.getBackupId()); + LOG.debug(" Backup directory: " + lastIncrImage.getRootDir()); + } + } + } + LOG.debug("Got " + ancestors.size() + " ancestors for the current backup."); + return ancestors; + } + + /** + * Get the direct ancestors of this backup for one table involved. + * @param backupContext backup context + * @param table table + * @return backupImages on the dependency list + * @throws BackupException exception + * @throws IOException exception + */ + protected ArrayList getAncestors(BackupContext backupContext, String table) + throws BackupException, IOException { + ArrayList ancestors = getAncestors(backupContext); + ArrayList tableAncestors = new ArrayList(); + for (BackupImage image : ancestors) { + if (image.hasTable(table)) { + tableAncestors.add(image); + if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) { + break; + } + } + } + return tableAncestors; + } + + /** + * hbase:backup operations + */ + + /** + * Updates status (state) of a backup session in a persistent store + * @param context context + * @throws IOException exception + */ + public void updateBackupStatus(BackupContext context) throws IOException { + systemTable.updateBackupStatus(context); + } + + /** + * Read the last backup start code (timestamp) of last successful backup. Will return null + * if there is no startcode stored in hbase:backup or the value is of length 0. These two + * cases indicate there is no successful backup completed so far. + * @return the timestamp of a last successful backup + * @throws IOException exception + */ + public String readBackupStartCode() throws IOException { + return systemTable.readBackupStartCode(); + } + + /** + * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte. + * @param startCode start code + * @throws IOException exception + */ + public void writeBackupStartCode(String startCode) throws IOException { + systemTable.writeBackupStartCode(startCode); + } + + /** + * Get the RS log information after the last log roll from hbase:backup. + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult() throws IOException { + return systemTable.readRegionServerLastLogRollResult(); + } + + /** + * Get all completed backup information (in desc order by time) + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public ArrayList getBackupHistory() throws IOException { + return systemTable.getBackupHistory(); + } + + /** + * Write the current timestamps for each regionserver to hbase:backup after a successful full or + * incremental backup. Each table may have a different set of log timestamps. The saved timestamp + * is of the last log file that was backed up already. + * @param tables tables + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, + HashMap newTimestamps) throws IOException { + systemTable.writeRegionServerLogTimestamp(tables, newTimestamps); + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. The info is stored for each table as a concatinated string on ZK + * under //hbase//backup//incr//tablelogtimestamp//table_name + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public HashMap> readLogTimestampMap() throws IOException { + return systemTable.readLogTimestampMap(); + } + + /** + * Return the current tables covered by incremental backup. + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet() throws IOException { + return systemTable.getIncrementalBackupTableSet(); + } + + /** + * Adds set of tables to overall incremental backup table set + * @param tables tables + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables) throws IOException { + systemTable.addIncrementalBackupTableSet(tables); + } + + /** + * Saves list of WAL files after incremental backup operation. These files will be stored until + * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be + * safely purged. + */ + + public void recordWALFiles(List files) throws IOException { + systemTable.addWALFiles(files, backupContext.getBackupId()); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java new file mode 100644 index 00000000000..f41540bc7bf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java @@ -0,0 +1,814 @@ +/** + * 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.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.FSUtils; +import org.codehaus.jackson.JsonGenerationException; +import org.codehaus.jackson.JsonParseException; +import org.codehaus.jackson.map.JsonMappingException; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled + * as manifest file together with data. So that each backup image will contain all the info needed + * for restore. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupManifest { + + private static final Log LOG = LogFactory.getLog(BackupManifest.class); + + // manifest file name + public static final String FILE_NAME = ".backup.manifest"; + + // manifest file version, current is 1.0 + public static final String MANIFEST_VERSION = "1.0"; + + // tags of fields for manifest file + public static final String TAG_VERSION = "Manifest-Version"; + public static final String TAG_BACKUPID = "Backup-Id"; + public static final String TAG_BACKUPTYPE = "Backup-Type"; + public static final String TAG_TABLESET = "Table-Set"; + public static final String TAG_STARTTS = "Start-Timestamp"; + public static final String TAG_COMPLETETS = "Complete-Timestamp"; + public static final String TAG_TABLEBYTES = "Total-Table-Bytes"; + public static final String TAG_LOGBYTES = "Total-Log-Bytes"; + public static final String TAG_INCRTIMERANGE = "Incremental-Time-Range"; + public static final String TAG_DEPENDENCY = "Dependency"; + public static final String TAG_IMAGESTATE = "Image-State"; + public static final String TAG_COMPACTION = "Compaction"; + + public static final String ERROR_DEPENDENCY = "DEPENDENCY_ERROR"; + + public static final int DELETE_SUCCESS = 0; + public static final int DELETE_FAILED = -1; + + // currently only one state, will have CONVERTED, and MERGED in future JIRA + public static final String IMAGE_STATE_ORIG = "ORIGINAL"; + public static final String IMAGE_STATE_CONVERT = "CONVERTED"; + public static final String IMAGE_STATE_MERGE = "MERGED"; + public static final String IMAGE_STATE_CONVERT_MERGE = "CONVERTED,MERGED"; + + // backup image, the dependency graph is made up by series of backup images + + public static class BackupImage implements Comparable { + + private String backupId; + private String type; + private String rootDir; + private String tableSet; + private long startTs; + private long completeTs; + private ArrayList ancestors; + + public BackupImage() { + super(); + } + + public BackupImage(String backupId, String type, String rootDir, String tableSet, long startTs, + long completeTs) { + this.backupId = backupId; + this.type = type; + this.rootDir = rootDir; + this.tableSet = tableSet; + this.startTs = startTs; + this.completeTs = completeTs; + } + + public String getBackupId() { + return backupId; + } + + public void setBackupId(String backupId) { + this.backupId = backupId; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getRootDir() { + return rootDir; + } + + public void setRootDir(String rootDir) { + this.rootDir = rootDir; + } + + public String getTableSet() { + return tableSet; + } + + public void setTableSet(String tableSet) { + this.tableSet = tableSet; + } + + public long getStartTs() { + return startTs; + } + + public void setStartTs(long startTs) { + this.startTs = startTs; + } + + public long getCompleteTs() { + return completeTs; + } + + public void setCompleteTs(long completeTs) { + this.completeTs = completeTs; + } + + public ArrayList getAncestors() { + if (this.ancestors == null) { + this.ancestors = new ArrayList(); + } + return this.ancestors; + } + + public void addAncestor(BackupImage backupImage) { + this.getAncestors().add(backupImage); + } + + public boolean hasAncestor(String token) { + for (BackupImage image : this.getAncestors()) { + if (image.getBackupId().equals(token)) { + return true; + } + } + return false; + } + + public boolean hasTable(String table) { + String[] tables = this.getTableSet().split(";"); + for (String t : tables) { + if (t.equals(table)) { + return true; + } + } + return false; + } + + @Override + public int compareTo(BackupImage other) { + String thisBackupId = this.getBackupId(); + String otherBackupId = other.getBackupId(); + Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1)); + Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1)); + return thisTS.compareTo(otherTS); + } + } + + // manifest version + private String version = MANIFEST_VERSION; + + // hadoop hbase configuration + protected Configuration config = null; + + // backup root directory + private String rootDir = null; + + // backup image directory + private String tableBackupDir = null; + + // backup log directory if this is an incremental backup + private String logBackupDir = null; + + // backup token + private String token; + + // backup type, full or incremental + private String type; + + // the table set for the backup + private ArrayList tableSet; + + // actual start timestamp of the backup process + private long startTs; + + // actual complete timestamp of the backup process + private long completeTs; + + // total bytes for table backup image + private long tableBytes; + + // total bytes for the backed-up logs for incremental backup + private long logBytes; + + // the region server timestamp for tables: + // > + private Map> incrTimeRanges; + + // dependency of this backup, including all the dependent images to do PIT recovery + private Map dependency; + + // the state of backup image + private String imageState; + + // the indicator of the image compaction + private boolean isCompacted = false; + + // the merge chain of the original backups, null if not a merged backup + private LinkedList mergeChain; + + /** + * Construct manifest for a ongoing backup. + * @param backupCtx The ongoing backup context + */ + public BackupManifest(BackupContext backupCtx) { + this.token = backupCtx.getBackupId(); + this.type = backupCtx.getType(); + this.rootDir = backupCtx.getTargetRootDir(); + if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + this.logBackupDir = backupCtx.getHLogTargetDir(); + this.logBytes = backupCtx.getTotalBytesCopied(); + } + this.startTs = backupCtx.getStartTs(); + this.completeTs = backupCtx.getEndTs(); + this.loadTableSet(backupCtx.getTableListAsString()); + this.setImageOriginal(); + } + + /** + * Construct a table level manifest for a backup of the named table. + * @param backupCtx The ongoing backup context + */ + public BackupManifest(BackupContext backupCtx, String table) { + this.token = backupCtx.getBackupId(); + this.type = backupCtx.getType(); + this.rootDir = backupCtx.getTargetRootDir(); + this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir(); + if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + this.logBackupDir = backupCtx.getHLogTargetDir(); + this.logBytes = backupCtx.getTotalBytesCopied(); + } + this.startTs = backupCtx.getStartTs(); + this.completeTs = backupCtx.getEndTs(); + this.loadTableSet(table); + this.setImageOriginal(); + } + + /** + * Construct manifest from a backup directory. + * @param conf configuration + * @param backupPath backup path + * @throws BackupException exception + */ + public BackupManifest(Configuration conf, Path backupPath) throws BackupException { + + LOG.debug("Loading manifest from: " + backupPath.toString()); + // The input backupDir may not exactly be the backup table dir. + // It could be the backup log dir where there is also a manifest file stored. + // This variable's purpose is to keep the correct and original location so + // that we can store/persist it. + this.tableBackupDir = backupPath.toString(); + this.config = conf; + try { + + FileSystem fs = backupPath.getFileSystem(conf); + FileStatus[] subFiles = FSUtils.listStatus(fs, backupPath); + if (subFiles == null) { + String errorMsg = backupPath.toString() + " does not exist"; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + for (FileStatus subFile : subFiles) { + if (subFile.getPath().getName().equals(FILE_NAME)) { + + // load and set manifest field from file content + FSDataInputStream in = fs.open(subFile.getPath()); + Properties props = new Properties(); + try { + props.load(in); + } catch (IOException e) { + LOG.error("Error when loading from manifest file!"); + throw e; + } finally { + in.close(); + } + + this.version = props.getProperty(TAG_VERSION); + this.token = props.getProperty(TAG_BACKUPID); + this.type = props.getProperty(TAG_BACKUPTYPE); + // Here the parameter backupDir is where the manifest file is. + // There should always be a manifest file under: + // backupRootDir/namespace/table/backupId/.backup.manifest + this.rootDir = backupPath.getParent().getParent().getParent().toString(); + + Path p = backupPath.getParent(); + if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) { + this.rootDir = p.getParent().toString(); + } else { + this.rootDir = p.getParent().getParent().toString(); + } + + this.loadTableSet(props.getProperty(TAG_TABLESET)); + + this.startTs = Long.parseLong(props.getProperty(TAG_STARTTS)); + this.completeTs = Long.parseLong(props.getProperty(TAG_COMPLETETS)); + this.tableBytes = Long.parseLong(props.getProperty(TAG_TABLEBYTES)); + if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + this.logBytes = (Long.parseLong(props.getProperty(TAG_LOGBYTES))); + LOG.debug("convert will be implemented by future jira"); + } + this.loadIncrementalTimeRanges(props.getProperty(TAG_INCRTIMERANGE)); + this.loadDependency(props.getProperty(TAG_DEPENDENCY)); + this.imageState = props.getProperty(TAG_IMAGESTATE); + this.isCompacted = + props.getProperty(TAG_COMPACTION).equalsIgnoreCase("TRUE") ? true : false; + LOG.debug("merge and from existing snapshot will be implemented by future jira"); + LOG.debug("Loaded manifest instance from manifest file: " + + FSUtils.getPath(subFile.getPath())); + return; + } + } + String errorMsg = "No manifest file found in: " + backupPath.toString(); + LOG.error(errorMsg); + throw new IOException(errorMsg); + + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * Load table set from a table set list string (t1;t2;t3;...). + * @param tableSetStr Table set list string + */ + private void loadTableSet(String tableSetStr) { + + LOG.debug("Loading table set: " + tableSetStr); + + String[] tableSet = tableSetStr.split(";"); + this.tableSet = this.getTableSet(); + if (this.tableSet.size() > 0) { + this.tableSet.clear(); + } + for (int i = 0; i < tableSet.length; i++) { + this.tableSet.add(tableSet[i]); + } + + LOG.debug(tableSet.length + " tables exist in table set."); + } + + public void setImageOriginal() { + this.imageState = IMAGE_STATE_ORIG; + } + + /** + * Get the table set of this image. + * @return The table set list + */ + public ArrayList getTableSet() { + if (this.tableSet == null) { + this.tableSet = new ArrayList(); + } + return this.tableSet; + } + + /** + * Persist the manifest file. + * @throws IOException IOException when storing the manifest file. + */ + public void store(Configuration conf) throws BackupException { + Properties props = new Properties(); + props.setProperty(TAG_VERSION, this.version); + props.setProperty(TAG_BACKUPID, this.token); + props.setProperty(TAG_BACKUPTYPE, this.type); + props.setProperty(TAG_TABLESET, this.getTableSetStr()); + LOG.debug("convert will be supported in future jira"); + // String convertedTables = this.getConvertedTableSetStr(); + // if (convertedTables != null ) + // props.setProperty(TAG_CONVERTEDTABLESET, convertedTables); + props.setProperty(TAG_STARTTS, Long.toString(this.startTs)); + props.setProperty(TAG_COMPLETETS, Long.toString(this.completeTs)); + props.setProperty(TAG_TABLEBYTES, Long.toString(this.tableBytes)); + if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + props.setProperty(TAG_LOGBYTES, Long.toString(this.logBytes)); + } + props.setProperty(TAG_INCRTIMERANGE, this.getIncrTimestampStr()); + props.setProperty(TAG_DEPENDENCY, this.getDependencyStr()); + props.setProperty(TAG_IMAGESTATE, this.getImageState()); + props.setProperty(TAG_COMPACTION, this.isCompacted ? "TRUE" : "FALSE"); + LOG.debug("merge will be supported in future jira"); + // props.setProperty(TAG_MERGECHAIN, this.getMergeChainStr()); + LOG.debug("backup from existing snapshot will be supported in future jira"); + // props.setProperty(TAG_FROMSNAPSHOT, this.isFromSnapshot() ? "TRUE" : "FALSE"); + + // write the file, overwrite if already exist + Path manifestFilePath = + new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir) + + File.separator + FILE_NAME); + try { + FSDataOutputStream out = manifestFilePath.getFileSystem(conf).create(manifestFilePath, true); + props.store(out, "HBase backup manifest."); + out.close(); + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + + LOG.debug("Manifest file stored to " + this.tableBackupDir != null ? this.tableBackupDir + : this.logBackupDir + File.separator + FILE_NAME); + } + + /** + * Get the table set string in the format of t1;t2;t3... + */ + private String getTableSetStr() { + return BackupUtil.concat(getTableSet(), ";"); + } + + public String getImageState() { + return imageState; + } + + public String getVersion() { + return version; + } + + /** + * Get this backup image. + * @return the backup image. + */ + public BackupImage getBackupImage() { + return this.getDependency().get(this.token); + } + + /** + * Add dependent backup image for this backup. + * @param image The direct dependent backup image + */ + public void addDependentImage(BackupImage image) { + this.getDependency().get(this.token).addAncestor(image); + this.setDependencyMap(this.getDependency(), image); + } + + /** + * Get the dependency' string in the json format. + */ + private String getDependencyStr() { + BackupImage thisImage = this.getDependency().get(this.token); + if (thisImage == null) { + LOG.warn("There is no dependency set yet."); + return null; + } + + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.writeValueAsString(thisImage); + } catch (JsonGenerationException e) { + LOG.error("Error when generating dependency string from backup image.", e); + return ERROR_DEPENDENCY; + } catch (JsonMappingException e) { + LOG.error("Error when generating dependency string from backup image.", e); + return ERROR_DEPENDENCY; + } catch (IOException e) { + LOG.error("Error when generating dependency string from backup image.", e); + return ERROR_DEPENDENCY; + } + } + + /** + * Get all dependent backup images. The image of this backup is also contained. + * @return The dependent backup images map + */ + public Map getDependency() { + if (this.dependency == null) { + this.dependency = new HashMap(); + LOG.debug(this.rootDir + " " + this.token + " " + this.type); + this.dependency.put(this.token, + new BackupImage(this.token, this.type, this.rootDir, this.getTableSetStr(), this.startTs, + this.completeTs)); + } + return this.dependency; + } + + /** + * Set the incremental timestamp map directly. + * @param incrTimestampMap timestamp map + */ + public void setIncrTimestampMap(HashMap> incrTimestampMap) { + this.incrTimeRanges = incrTimestampMap; + } + + /** + * Get the incremental time range string in the format of: + * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,... + */ + private String getIncrTimestampStr() { + StringBuilder sb = new StringBuilder(); + for (Entry> tableEntry : this.getIncrTimestamps().entrySet()) { + sb.append(tableEntry.getKey() + ","); // table + for (Entry rsEntry : tableEntry.getValue().entrySet()) { + sb.append(rsEntry.getKey() + ":"); // region server + sb.append(rsEntry.getValue() + ","); // timestamp + } + if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ',') { + sb.deleteCharAt(sb.length() - 1); + } + sb.append(";"); + } + if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ';') { + sb.deleteCharAt(sb.length() - 1); + } + return sb.toString(); + } + + public Map> getIncrTimestamps() { + if (this.incrTimeRanges == null) { + this.incrTimeRanges = new HashMap>(); + } + return this.incrTimeRanges; + } + + /** + * Load incremental timestamps from a given string, and store them in the collection. The + * timestamps in string is in the format of + * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,... + * @param timeRangesInStr Incremental time ranges in string + */ + private void loadIncrementalTimeRanges(String timeRangesStr) throws IOException { + + LOG.debug("Loading table's incremental time ranges of region servers from string in manifest: " + + timeRangesStr); + + Map> timeRangeMap = this.getIncrTimestamps(); + + String[] entriesOfTables = timeRangesStr.split(";"); + for (int i = 0; i < entriesOfTables.length; i++) { + String[] itemsForTable = entriesOfTables[i].split(","); + + // validate the incremental timestamps string format for a table: + // t1,rs1:ts,rs2:ts,... + if (itemsForTable.length < 1) { + String errorMsg = "Wrong incremental time range format: " + timeRangesStr; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + HashMap rsTimestampMap = new HashMap(); + for (int j = 1; j < itemsForTable.length; j++) { + String[] rsTsEntry = itemsForTable[j].split(":"); + + // validate the incremental timestamps string format for a region server: + // rs1:ts + if (rsTsEntry.length != 2) { + String errorMsg = "Wrong incremental timestamp format: " + itemsForTable[j]; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + // an entry for timestamp of a region server + rsTimestampMap.put(rsTsEntry[0], rsTsEntry[1]); + } + + timeRangeMap.put(itemsForTable[0], rsTimestampMap); + } + + // all entries have been loaded + LOG.debug(entriesOfTables.length + " tables' incremental time ranges have been loaded."); + } + + /** + * 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(); + for (BackupImage image : this.getDependency().values()) { + 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(String table) { + ArrayList tableImageList = new ArrayList(); + ArrayList imageList = getRestoreDependentList(true); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_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(String table) { + ArrayList tableImageList = new ArrayList(); + ArrayList imageList = getRestoreDependentList(false); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + } + } + return tableImageList; + } + + /** + * Load dependency from a dependency json string. + * @param dependencyStr The dependency string + * @throws IOException exception + */ + private void loadDependency(String dependencyStr) throws IOException { + + LOG.debug("Loading dependency: " + dependencyStr); + + String msg = "Dependency is broken in the manifest."; + if (dependencyStr.equals(ERROR_DEPENDENCY)) { + throw new IOException(msg); + } + + ObjectMapper mapper = new ObjectMapper(); + BackupImage image = null; + try { + image = mapper.readValue(dependencyStr, BackupImage.class); + } catch (JsonParseException e) { + LOG.error(msg); + throw new IOException(e.getMessage()); + } catch (JsonMappingException e) { + LOG.error(msg); + throw new IOException(e.getMessage()); + } catch (IOException e) { + LOG.error(msg); + throw new IOException(e.getMessage()); + } + LOG.debug("Manifest's current backup image information:"); + LOG.debug(" Token: " + image.getBackupId()); + LOG.debug(" Backup directory: " + image.getRootDir()); + this.setDependencyMap(this.getDependency(), image); + + LOG.debug("Dependent images map:"); + for (Entry entry : this.getDependency().entrySet()) { + LOG.debug(" " + entry.getKey() + " : " + entry.getValue().getBackupId() + " -- " + + entry.getValue().getRootDir()); + } + + LOG.debug("Dependency has been loaded."); + } + + /** + * Recursively set the dependency map of the backup images. + * @param map The dependency map + * @param image The backup image + */ + private void setDependencyMap(Map map, BackupImage image) { + if (image == null) { + return; + } else { + map.put(image.getBackupId(), image); + for (BackupImage img : image.getAncestors()) { + setDependencyMap(map, img); + } + } + } + + /** + * Check whether backup image1 could cover backup image2 or not. + * @param image1 backup image 1 + * @param image2 backup image 2 + * @return true if image1 can cover image2, otherwise false + */ + public static boolean canCoverImage(BackupImage image1, BackupImage image2) { + // image1 can cover image2 only when the following conditions are satisfied: + // - image1 must not be an incremental image; + // - image1 must be taken after image2 has been taken; + // - table set of image1 must cover the table set of image2. + if (image1.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + return false; + } + if (image1.getStartTs() < image2.getStartTs()) { + return false; + } + String[] image1TableSet = image1.getTableSet().split(";"); + String[] image2TableSet = image2.getTableSet().split(";"); + boolean found = false; + for (int i = 0; i < image2TableSet.length; i++) { + found = false; + for (int j = 0; j < image1TableSet.length; j++) { + if (image2TableSet[i].equals(image1TableSet[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().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) { + return false; + } + if (image1.getStartTs() < image.getStartTs()) { + return false; + } + } + + ArrayList image1TableSet = new ArrayList(); + for (BackupImage image1 : fullImages) { + String[] tableSet = image1.getTableSet().split(";"); + for (String table : tableSet) { + image1TableSet.add(table); + } + } + ArrayList image2TableSet = new ArrayList(); + String[] tableSet = image.getTableSet().split(";"); + for (String table : tableSet) { + image2TableSet.add(table); + } + + for (int i = 0; i < image2TableSet.size(); i++) { + if (image1TableSet.contains(image2TableSet.get(i)) == false) { + return false; + } + } + + LOG.debug("Full image set can cover image " + image.getBackupId()); + return true; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java new file mode 100644 index 00000000000..9610b270341 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -0,0 +1,66 @@ +/** + * 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.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * HConstants holds a bunch of HBase Backup and Restore constants + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public final class BackupRestoreConstants { + + // constants for znode data keys in backup znode + public static final String BACKUP_PROGRESS = "progress"; + public static final String BACKUP_START_TIME = "startTs"; + public static final String BACKUP_INPROGRESS_PHASE = "phase"; + public static final String BACKUP_COMPLETE_TIME = "completeTs"; + public static final String BACKUP_FAIL_TIME = "failedTs"; + public static final String BACKUP_FAIL_PHASE = "failedphase"; + public static final String BACKUP_FAIL_MSG = "failedmessage"; + public static final String BACKUP_ROOT_PATH = "targetRootDir"; + public static final String BACKUP_REQUEST_TABLE_LIST = "tablelist"; + public static final String BACKUP_REQUEST_TYPE = "type"; + public static final String BACKUP_BYTES_COPIED = "bytescopied"; + public static final String BACKUP_ANCESTORS = "ancestors"; + public static final String BACKUP_EXISTINGSNAPSHOT = "snapshot"; + + public static final String BACKUP_TYPE_FULL = "full"; + public static final String BACKUP_TYPE_INCR = "incremental"; + + // delimiter in tablename list in restore command + public static final String TABLENAME_DELIMITER_IN_COMMAND = ","; + + // delimiter in znode data + public static final String ZNODE_DATA_DELIMITER = ","; + + public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root"; + + public static final String BACKUPID_PREFIX = "backup_"; + + public static enum BACKUP_COMMAND { + CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, + } + + private BackupRestoreConstants() { + // Can't be instantiated with this ctor. + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java new file mode 100644 index 00000000000..1c38cf40e9e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java @@ -0,0 +1,61 @@ +/** + * 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.MapReduceBackupCopyService; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreService; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.util.ReflectionUtils; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class BackupRestoreServiceFactory { + + public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class"; + public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class"; + + private BackupRestoreServiceFactory(){ + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Gets incremental restore service + * @param conf - configuration + * @return incremental backup service instance + */ + public static IncrementalRestoreService getIncrementalRestoreService(Configuration conf) { + Class cls = + conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreService.class, + IncrementalRestoreService.class); + return ReflectionUtils.newInstance(cls, conf); + } + + /** + * Gets backup copy service + * @param conf - configuration + * @return backup copy service + */ + public static BackupCopyService getBackupCopyService(Configuration conf) { + Class cls = + conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyService.class, + BackupCopyService.class); + return ReflectionUtils.newInstance(cls, conf); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java new file mode 100644 index 00000000000..c26e1d45e9f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.Serializable; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * Backup status and related information encapsulated for a table. + * At this moment only TargetDir and SnapshotName is encapsulated here. + * future Jira will be implemented for progress, bytesCopies, phase, etc. + */ + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class BackupStatus implements Serializable { + + private static final long serialVersionUID = -5968397963548535982L; + + // table name for backup + private String table; + + // target directory of the backup image for this table + private String targetDir; + + // snapshot name for offline/online snapshot + private String snapshotName = null; + + public BackupStatus(String table, String targetRootDir, String backupId) { + this.table = table; + this.targetDir = HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); + } + + public String getSnapshotName() { + return snapshotName; + } + + public void setSnapshotName(String snapshotName) { + this.snapshotName = snapshotName; + } + + public String getTargetDir() { + return targetDir; + } + + public String getTable() { + return table; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java new file mode 100644 index 00000000000..14769f9d623 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java @@ -0,0 +1,642 @@ +/** + * 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.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS; +import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; + +/** + * This class provides 'hbase:backup' table API + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class BackupSystemTable { + + private static final Log LOG = LogFactory.getLog(BackupSystemTable.class); + private final static String TABLE_NAMESPACE = "hbase"; + private final static String TABLE_NAME = "backup"; + private final static TableName tableName = TableName.valueOf(TABLE_NAMESPACE, TABLE_NAME); + public final static byte[] familyName = "f".getBytes(); + + // Connection to HBase cluster + private static Connection connection; + // Cluster configuration + private static Configuration config; + // singleton + private static BackupSystemTable table; + + /** + * Get instance by a given configuration + * @param conf - HBase configuration + * @return instance of BackupSystemTable + * @throws IOException exception + */ + public synchronized static BackupSystemTable getTable(Configuration conf) throws IOException { + if (connection == null) { + connection = ConnectionFactory.createConnection(conf); + config = conf; + // Verify hbase:system exists + createSystemTableIfNotExists(); + table = new BackupSystemTable(); + } + return table; + } + + /** + * TODO: refactor + * @throws IOException exception + */ + public static void close() throws IOException { + connection.close(); + table = null; + } + + /** + * Gets table name + * @return table name + */ + public static TableName getTableName() { + return tableName; + } + + private static void createSystemTableIfNotExists() throws IOException { + Admin admin = null; + try { + admin = connection.getAdmin(); + if (admin.tableExists(tableName) == false) { + HTableDescriptor tableDesc = new HTableDescriptor(tableName); + HColumnDescriptor colDesc = new HColumnDescriptor(familyName); + colDesc.setMaxVersions(1); + int ttl = + config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT); + colDesc.setTimeToLive(ttl); + tableDesc.addFamily(colDesc); + admin.createTable(tableDesc); + } + } catch (IOException e) { + LOG.error(e); + throw e; + } finally { + if (admin != null) { + admin.close(); + } + } + } + + private BackupSystemTable() { + } + + /** + * Updates status (state) of a backup session in hbase:backup table + * @param context context + * @throws IOException exception + */ + public void updateBackupStatus(BackupContext context) throws IOException { + + if (LOG.isDebugEnabled()) { + LOG.debug("update backup status in hbase:backup for: " + context.getBackupId() + + " set status=" + context.getFlag()); + } + Table table = null; + try { + table = connection.getTable(tableName); + Put put = BackupSystemTableHelper.createPutForBackupContext(context); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Deletes backup status from hbase:backup table + * @param backupId backup id + * @throws IOException exception + */ + + public void deleteBackupStatus(String backupId) throws IOException { + + if (LOG.isDebugEnabled()) { + LOG.debug("delete backup status in hbase:backup for " + backupId); + } + Table table = null; + try { + table = connection.getTable(tableName); + Delete del = BackupSystemTableHelper.createDeletForBackupContext(backupId); + table.delete(del); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Reads backup status object (instance of BackupContext) from hbase:backup table + * @param backupId - backupId + * @return Current status of backup session or null + */ + + public BackupContext readBackupStatus(String backupId) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("read backup status from hbase:backup for: " + backupId); + } + + Table table = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForBackupContext(backupId); + Result res = table.get(get); + if(res.isEmpty()){ + return null; + } + return BackupSystemTableHelper.resultToBackupContext(res); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * 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. + * @return the timestamp of last successful backup + * @throws IOException exception + */ + public String readBackupStartCode() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("read backup start code from hbase:backup"); + } + Table table = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForStartCode(); + 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); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte. + * @param startCode start code + * @throws IOException exception + */ + public void writeBackupStartCode(String startCode) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("write backup start code to hbase:backup " + startCode); + } + Table table = null; + try { + table = connection.getTable(tableName); + Put put = BackupSystemTableHelper.createPutForStartCode(startCode); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Get the Region Servers log information after the last log roll from hbase:backup. + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult() + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("read region server last roll log result to hbase:backup"); + } + Table table = null; + ResultScanner scanner = null; + + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult(); + scan.setMaxVersions(1); + scanner = table.getScanner(scan); + Result res = null; + HashMap rsTimestampMap = new HashMap(); + while ((res = scanner.next()) != null) { + res.advance(); + Cell cell = res.current(); + byte[] row = CellUtil.cloneRow(cell); + String server = + BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row); + + byte[] data = CellUtil.cloneValue(cell); + rsTimestampMap.put(server, new String(data)); + } + return rsTimestampMap; + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } + + /** + * Writes Region Server last roll log result (timestamp) to hbase:backup table + * @param server - Region Server name + * @param fileName - last log timestamp + * @throws IOException exception + */ + public void writeRegionServerLastLogRollResult(String server, String fileName) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("write region server last roll log result to hbase:backup"); + } + Table table = null; + try { + table = connection.getTable(tableName); + Put put = + BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, fileName); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Get all completed backup information (in desc order by time) + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public ArrayList getBackupHistory() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("get backup history from hbase:backup"); + } + Table table = null; + ResultScanner scanner = null; + ArrayList list = new ArrayList(); + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForBackupHistory(); + scan.setMaxVersions(1); + scanner = table.getScanner(scan); + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current()); + if (context.getFlag() != BACKUPSTATUS.COMPLETE) { + continue; + } + + BackupCompleteData history = new BackupCompleteData(); + history.setBackupToken(context.getBackupId()); + history.setStartTime(Long.toString(context.getStartTs())); + history.setEndTime(Long.toString(context.getEndTs())); + history.setBackupRootPath(context.getTargetRootDir()); + history.setTableList(context.getTableListAsString()); + history.setType(context.getType()); + history.setBytesCopied(Long.toString(context.getTotalBytesCopied())); + + if (context.fromExistingSnapshot()) { + history.markFromExistingSnapshot(); + } + list.add(history); + } + return BackupUtil.sortHistoryListDesc(list); + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } + + /** + * Get all backup session with a given status (in desc order by time) + * @param status status + * @return history info of backup contexts + * @throws IOException exception + */ + public ArrayList getBackupContexts(BACKUPSTATUS status) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("get backup contexts from hbase:backup"); + } + Table table = null; + ResultScanner scanner = null; + ArrayList list = new ArrayList(); + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForBackupHistory(); + scan.setMaxVersions(1); + scanner = table.getScanner(scan); + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current()); + if (context.getFlag() != status){ + continue; + } + list.add(context); + } + return list; + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } + + /** + * Write the current timestamps for each regionserver to hbase:backup after a successful full or + * incremental backup. The saved timestamp is of the last log file that was backed up already. + * @param tables tables + * @param newTimestamps timestamps + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, + HashMap newTimestamps) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("write RS log ts to HBASE_BACKUP"); + } + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : newTimestamps.entrySet()) { + String host = entry.getKey(); + String timestamp = entry.getValue(); + sb.append(host).append(BackupUtil.FIELD_SEPARATOR).append(timestamp) + .append(BackupUtil.RECORD_SEPARATOR); + } + String smap = sb.toString(); + List puts = new ArrayList(); + for (String table : tables) { + Put put = BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smap); + puts.add(put); + } + Table table = null; + try { + table = connection.getTable(tableName); + table.put(puts); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. The info is stored for each table as a concatenated string of + * rs->timestapmp + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public HashMap> readLogTimestampMap() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("read RS log ts from HBASE_BACKUP"); + } + + Table table = null; + ResultScanner scanner = null; + HashMap> tableTimestampMap = + new HashMap>(); + + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap(); + scanner = table.getScanner(scan); + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + Cell cell = res.current(); + byte[] row = CellUtil.cloneRow(cell); + String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row); + HashMap lastBackup = new HashMap(); + byte[] data = CellUtil.cloneValue(cell); + if (data == null) { + // TODO + throw new IOException("Data of last backup data from HBASE_BACKUP " + + "is empty. Create a backup first."); + } + if (data != null && data.length > 0) { + String s = new String(data); + String[] records = s.split(BackupUtil.RECORD_SEPARATOR); + for (String record : records) { + String[] flds = record.split(BackupUtil.FIELD_SEPARATOR); + if (flds.length != 2) { + throw new IOException("data from HBASE_BACKUP is corrupted: " + + Arrays.toString(flds)); + } + lastBackup.put(flds[0], flds[1]); + } + tableTimestampMap.put(tabName, lastBackup); + } + } + return tableTimestampMap; + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } + + /** + * Return the current tables covered by incremental backup. + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("get incr backup table set from hbase:backup"); + } + Table table = null; + TreeSet set = new TreeSet(); + + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet(); + 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 + // TODO ns:table as qualifier? + set.add(new String(CellUtil.cloneQualifier(cell))); + } + return set; + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Add tables to global incremental backup set + * @param tables - set of tables + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("add incr backup table set to hbase:backup"); + } + Table table = null; + try { + table = connection.getTable(tableName); + Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Register WAL files as eligible for deletion + * @param files files + * @throws IOException exception + */ + public void addWALFiles(List files, String backupId) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("add WAL files to hbase:backup"); + } + Table table = null; + try { + table = connection.getTable(tableName); + List puts = BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId); + table.put(puts); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Check if WAL file is eligible for deletion + * @param file file + * @return true, if - yes. + * @throws IOException exception + */ + public boolean checkWALFile(String file) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Check if WAL file has been already backuped in hbase:backup"); + } + Table table = null; + try { + table = connection.getTable(tableName); + Get get = BackupSystemTableHelper.createGetForCheckWALFile(file); + Result res = table.get(get); + if (res.isEmpty()){ + return false; + } + return true; + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Checks if we have at least one backup session in hbase:backup This API is used by + * BackupLogCleaner + * @return true, if - at least one session exists in hbase:backup table + * @throws IOException exception + */ + public boolean hasBackupSessions() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("has backup sessions from hbase:backup"); + } + Table table = null; + ResultScanner scanner = null; + boolean result = false; + try { + table = connection.getTable(tableName); + Scan scan = BackupSystemTableHelper.createScanForBackupHistory(); + scan.setMaxVersions(1); + scan.setCaching(1); + scanner = table.getScanner(scan); + if (scanner.next() != null) { + result = true; + } + return result; + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java new file mode 100644 index 00000000000..bf62a84382c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java @@ -0,0 +1,314 @@ +/** + * 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.Arrays; +import java.util.List; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +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.Scan; + + +/** + * A collection for methods used by BackupSystemTable. + */ + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class BackupSystemTableHelper { + + /** + * hbase:backup schema: + * 1. Backup sessions rowkey= "session." + backupId; value = serialized + * BackupContext + * 2. Backup start code rowkey = "startcode"; value = startcode + * 3. Incremental backup set rowkey="incrbackupset"; value=[list of tables] + * 4. Table-RS-timestamp map rowkey="trslm"+ table_name; value = map[RS-> last WAL timestamp] + * 5. RS - WAL ts map rowkey="rslogts."+server; value = last WAL timestamp + * 6. WALs recorded rowkey="wals."+WAL unique file name; value = NULL (value is not used) + */ + private static final Log LOG = LogFactory.getLog(BackupSystemTableHelper.class); + + private final static String BACKUP_CONTEXT_PREFIX = "session."; + private final static String START_CODE_ROW = "startcode"; + 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 WALS_PREFIX = "wals."; + + private final static byte[] q0 = "0".getBytes(); + private final static byte[] EMPTY_VALUE = new byte[] {}; + + private BackupSystemTableHelper() { + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Creates Put operation for a given backup context object + * @param context backup context + * @return put operation + * @throws IOException exception + */ + static Put createPutForBackupContext(BackupContext context) throws IOException { + + Put put = new Put((BACKUP_CONTEXT_PREFIX + context.getBackupId()).getBytes()); + put.addColumn(BackupSystemTable.familyName, q0, context.toByteArray()); + return put; + } + + /** + * Creates Get operation for a given backup id + * @param backupId - backup's ID + * @return get operation + * @throws IOException exception + */ + static Get createGetForBackupContext(String backupId) throws IOException { + Get get = new Get((BACKUP_CONTEXT_PREFIX + backupId).getBytes()); + get.addFamily(BackupSystemTable.familyName); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Delete operation for a given backup id + * @param backupId - backup's ID + * @return delete operation + * @throws IOException exception + */ + public static Delete createDeletForBackupContext(String backupId) { + Delete del = new Delete((BACKUP_CONTEXT_PREFIX + backupId).getBytes()); + del.addFamily(BackupSystemTable.familyName); + return del; + } + + /** + * Converts Result to BackupContext + * @param res - HBase result + * @return backup context instance + * @throws IOException exception + */ + static BackupContext resultToBackupContext(Result res) throws IOException { + res.advance(); + Cell cell = res.current(); + return cellToBackupContext(cell); + } + + /** + * Creates Get operation to retrieve start code from hbase:backup + * @return get operation + * @throws IOException exception + */ + static Get createGetForStartCode() throws IOException { + Get get = new Get(START_CODE_ROW.getBytes()); + get.addFamily(BackupSystemTable.familyName); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Put operation to store start code to hbase:backup + * @return put operation + * @throws IOException exception + */ + static Put createPutForStartCode(String startCode) { + Put put = new Put(START_CODE_ROW.getBytes()); + put.addColumn(BackupSystemTable.familyName, q0, startCode.getBytes()); + return put; + } + + /** + * Creates Get to retrieve incremental backup table set from hbase:backup + * @return get operation + * @throws IOException exception + */ + static Get createGetForIncrBackupTableSet() throws IOException { + Get get = new Get(INCR_BACKUP_SET.getBytes()); + get.addFamily(BackupSystemTable.familyName); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Put to store incremental backup table set + * @param tables tables + * @return put operation + */ + static Put createPutForIncrBackupTableSet(Set tables) { + Put put = new Put(INCR_BACKUP_SET.getBytes()); + for (String table : tables) { + put.addColumn(BackupSystemTable.familyName, table.getBytes(), EMPTY_VALUE); + } + return put; + } + + /** + * Creates Scan operation to load backup history + * @return scan operation + */ + static Scan createScanForBackupHistory() { + Scan scan = new Scan(); + byte[] startRow = BACKUP_CONTEXT_PREFIX.getBytes(); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.familyName); + + return scan; + } + + /** + * Converts cell to backup context instance. + * @param current - cell + * @return backup context instance + * @throws IOException exception + */ + static BackupContext cellToBackupContext(Cell current) throws IOException { + byte[] data = CellUtil.cloneValue(current); + try { + BackupContext ctxt = BackupContext.fromByteArray(data); + return ctxt; + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + /** + * Creates Put to write RS last roll log timestamp map + * @param table - table + * @param smap - map, containing RS:ts + * @return put operation + */ + static Put createPutForWriteRegionServerLogTimestamp(String table, String smap) { + Put put = new Put((TABLE_RS_LOG_MAP_PREFIX + table).getBytes()); + put.addColumn(BackupSystemTable.familyName, q0, smap.getBytes()); + return put; + } + + /** + * Creates Scan to load table-> { RS -> ts} map of maps + * @return scan operation + */ + static Scan createScanForReadLogTimestampMap() { + Scan scan = new Scan(); + byte[] startRow = TABLE_RS_LOG_MAP_PREFIX.getBytes(); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.familyName); + + return scan; + } + + /** + * Get table name from rowkey + * @param cloneRow rowkey + * @return table name + */ + static String getTableNameForReadLogTimestampMap(byte[] cloneRow) { + int prefixSize = TABLE_RS_LOG_MAP_PREFIX.length(); + return new String(cloneRow, prefixSize, cloneRow.length - prefixSize); + } + + /** + * Creates Put to store RS last log result + * @param server - server name + * @param fileName - log roll result (timestamp) + * @return put operation + */ + static Put createPutForRegionServerLastLogRollResult(String server, String fileName) { + Put put = new Put((RS_LOG_TS_PREFIX + server).getBytes()); + put.addColumn(BackupSystemTable.familyName, q0, fileName.getBytes()); + return put; + } + + /** + * Creates Scan operation to load last RS log roll results + * @return scan operation + */ + static Scan createScanForReadRegionServerLastLogRollResult() { + Scan scan = new Scan(); + byte[] startRow = RS_LOG_TS_PREFIX.getBytes(); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.familyName); + + return scan; + } + + /** + * Get server's name from rowkey + * @param row - rowkey + * @return server's name + */ + static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) { + int prefixSize = RS_LOG_TS_PREFIX.length(); + return new String(row, prefixSize, row.length - prefixSize); + } + + /** + * Creates put list for list of WAL files + * @param files list of WAL file paths + * @param backupId backup id + * @return put list + * @throws IOException exception + */ + public static List createPutsForAddWALFiles(List files, String backupId) + throws IOException { + + List puts = new ArrayList(); + for (String file : files) { + LOG.debug("+++ put: " + BackupUtil.getUniqueWALFileNamePart(file)); + byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes(); + Put put = new Put(row); + put.addColumn(BackupSystemTable.familyName, q0, backupId.getBytes()); + puts.add(put); + } + return puts; + } + + /** + * Creates Get operation for a given wal file name + * @param file file + * @return get operation + * @throws IOException exception + */ + public static Get createGetForCheckWALFile(String file) throws IOException { + byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes(); + Get get = new Get(row); + get.addFamily(BackupSystemTable.familyName); + get.setMaxVersions(1); + return get; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java new file mode 100644 index 00000000000..ff8bd2e1b1c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java @@ -0,0 +1,564 @@ +/** + * 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.FileNotFoundException; +import java.io.IOException; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.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.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; + +/** + * A collection for methods used by multiple classes to backup HBase tables. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class BackupUtil { + protected static final Log LOG = LogFactory.getLog(BackupUtil.class); + + public static final String FIELD_SEPARATOR = "\001"; + public static final String RECORD_SEPARATOR = "\002"; + public static final String LOGNAME_SEPARATOR = "."; + protected static final String HDFS = "hdfs://"; + protected static Configuration conf = null; + + private BackupUtil(){ + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Set the configuration from a given one. + * @param newConf A new given configuration + */ + public synchronized static void setConf(Configuration newConf) { + conf = newConf; + } + + /** + * Get and merge Hadoop and HBase configuration. + * @throws IOException exception + */ + protected static Configuration getConf() { + if (conf == null) { + conf = new Configuration(); + HBaseConfiguration.merge(conf, HBaseConfiguration.create()); + } + return conf; + } + + /** + * 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 + */ + protected static HashMap getRSLogTimestampMins( + HashMap> rsLogTimestampMap) { + + if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) { + return null; + } + + HashMap rsLogTimestamptMins = new HashMap(); + HashMap> rsLogTimestampMapByRS = + new HashMap>(); + + for (Entry> tableEntry : rsLogTimestampMap.entrySet()) { + String table = tableEntry.getKey(); + HashMap rsLogTimestamp = tableEntry.getValue(); + for (Entry rsEntry : rsLogTimestamp.entrySet()) { + String rs = rsEntry.getKey(); + String ts = rsEntry.getValue(); + if (!rsLogTimestampMapByRS.containsKey(rs)) { + rsLogTimestampMapByRS.put(rs, new HashMap()); + rsLogTimestampMapByRS.get(rs).put(table, ts); + } else { + rsLogTimestampMapByRS.get(rs).put(table, ts); + } + } + } + + for (String rs : rsLogTimestampMapByRS.keySet()) { + rsLogTimestamptMins.put(rs, getMinValue(rsLogTimestampMapByRS.get(rs))); + } + + return rsLogTimestamptMins; + } + + /** + * Get the min value for all the Values a map. + * @param map map + * @return the min value + */ + protected static String getMinValue(HashMap map) { + String minTimestamp = null; + if (map != null) { + ArrayList timestampList = new ArrayList(map.values()); + Collections.sort(timestampList, new Comparator() { + @Override + public int compare(String s1, String s2) { + long l1 = Long.valueOf(s1); + long l2 = Long.valueOf(s2); + if (l1 > l2) { + return 1; + } else if (l1 < l2) { + return -1; + } else { + return 0; + } + } + }); + // The min among all the RS log timestamps will be kept in ZK. + minTimestamp = timestampList.get(0); + } + return minTimestamp; + } + + /** + * copy out Table RegionInfo into incremental backup image need to consider move this logic into + * HBackupFileSystem + * @param backupContext backup context + * @param conf configuration + * @throws IOException exception + * @throws InterruptedException exception + */ + protected static void copyTableRegionInfo(BackupContext backupContext, Configuration conf) + throws IOException, InterruptedException { + + Path rootDir = FSUtils.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 + for (String table : backupContext.getTables()) { + + LOG.debug("Attempting to copy table info for:" + table); + TableDescriptor orig = + FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, TableName.valueOf(table)); + + // write a copy of descriptor to the target directory + Path target = new Path(backupContext.getBackupStatus(table).getTargetDir()); + FileSystem targetFs = target.getFileSystem(conf); + FSTableDescriptors descriptors = + new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf)); + descriptors.createTableDescriptorForTableDirectory(target, orig, false); + LOG.debug("Finished copying tableinfo."); + + HBaseAdmin hbadmin = null; + // TODO: optimize + Connection conn = null; + List regions = null; + try { + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + regions = hbadmin.getTableRegions(TableName.valueOf(table)); + } catch (Exception e) { + throw new BackupException(e); + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if(conn != null){ + conn.close(); + } + } + + // For each region, write the region info to disk + LOG.debug("Starting to write region info for table " + table); + for (HRegionInfo regionInfo : regions) { + Path regionDir = + HRegion.getRegionDir(new Path(backupContext.getBackupStatus(table).getTargetDir()), + regionInfo); + regionDir = + new Path(backupContext.getBackupStatus(table).getTargetDir(), 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, HRegionInfo regionInfo) throws IOException { + final byte[] content = regionInfo.toDelimitedByteArray(); + Path regionInfoFile = new Path(regionInfoDir, ".regioninfo"); + // First check to get the permissions + FsPermission perms = FSUtils.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(); + } + } + + /** + * TODO: verify the code + * @param p path + * @return host name + * @throws IOException exception + */ + protected static String parseHostFromOldLog(Path p) throws IOException { + String n = p.getName(); + int idx = n.lastIndexOf(LOGNAME_SEPARATOR); + String s = URLDecoder.decode(n.substring(0, idx), "UTF8"); + return ServerName.parseHostname(s); + } + + public static String parseHostNameFromLogFile(Path p) throws IOException { + if (isArchivedLogFile(p)) { + return parseHostFromOldLog(p); + } else { + return DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname(); + } + } + + private static boolean isArchivedLogFile(Path p) { + String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR; + return p.toString().contains(oldLog); + } + + /** + * Return WAL file name + * @param walFileName WAL file name + * @return WAL file name + * @throws IOException exception + * @throws IllegalArgumentException exception + */ + public static String getUniqueWALFileNamePart(String walFileName) throws IOException { + return new Path(walFileName).getName(); + } + + /** + * Return WAL file name + * @param p - WAL file path + * @return WAL file name + * @throws IOException exception + */ + public static String getUniqueWALFileNamePart(Path p) throws IOException { + return p.getName(); + } + + /** + * 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 + */ + protected static String getCreationTime(Path p, Configuration conf) 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 ts; + } + + /** + * 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 = FSUtils.listStatus(fs, dir); + if (files != null) { + for (FileStatus fileStatus : files) { + if (fileStatus.isDir()) { + totalLength += getFilesLength(fs, fileStatus.getPath()); + } else { + totalLength += fileStatus.getLen(); + } + } + } + return totalLength; + } + + /** + * Keep the record for dependency for incremental backup and history info p.s, we may be able to + * merge this class into backupImage class later + */ + public static class BackupCompleteData implements Comparable { + private String startTime; + private String endTime; + private String type; + private String backupRootPath; + private String tableList; + private String backupToken; + private String bytesCopied; + private List ancestors; + private boolean fromExistingSnapshot = false; + + public List getAncestors() { + if (fromExistingSnapshot) { + return null; + } + if (this.ancestors == null) { + this.ancestors = new ArrayList(); + } + return this.ancestors; + } + + public void addAncestor(String backupToken) { + this.getAncestors().add(backupToken); + } + + public String getBytesCopied() { + return bytesCopied; + } + + public void setBytesCopied(String bytesCopied) { + this.bytesCopied = bytesCopied; + } + + public String getBackupToken() { + return backupToken; + } + + public void setBackupToken(String backupToken) { + this.backupToken = backupToken; + } + + public String getStartTime() { + return startTime; + } + + public void setStartTime(String startTime) { + this.startTime = startTime; + } + + public String getEndTime() { + return endTime; + } + + public void setEndTime(String endTime) { + this.endTime = endTime; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getBackupRootPath() { + return backupRootPath; + } + + public void setBackupRootPath(String backupRootPath) { + this.backupRootPath = backupRootPath; + } + + public String getTableList() { + return tableList; + } + + public void setTableList(String tableList) { + this.tableList = tableList; + } + + public boolean fromExistingSnapshot() { + return this.fromExistingSnapshot; + } + + public void markFromExistingSnapshot() { + this.fromExistingSnapshot = true; + } + + @Override + public int compareTo(BackupCompleteData o) { + Long thisTS = + new Long(this.getBackupToken().substring(this.getBackupToken().lastIndexOf("_") + 1)); + Long otherTS = + new Long(o.getBackupToken().substring(o.getBackupToken().lastIndexOf("_") + 1)); + return thisTS.compareTo(otherTS); + } + + } + + /** + * 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 (BackupCompleteData h : historyList) { + map.put(h.getStartTime(), h); + } + Iterator i = map.descendingKeySet().iterator(); + while (i.hasNext()) { + list.add(map.get(i.next())); + } + return list; + } + + /** + * Get list of all WAL files (WALs and archive) + * @param c - configuration + * @return list of WAL files + * @throws IOException exception + */ + public static List getListOfWALFiles(Configuration c) throws IOException { + Path rootDir = FSUtils.getRootDir(c); + Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); + List logFiles = new ArrayList(); + + FileSystem fs = FileSystem.get(c); + logFiles = getFiles(fs, logDir, logFiles, null); + logFiles = getFiles(fs, oldLogDir, logFiles, null); + return logFiles; + } + + /** + * Get list of all WAL files (WALs and archive) + * @param c - configuration + * @return list of WAL files + * @throws IOException exception + */ + public static List getListOfWALFiles(Configuration c, PathFilter filter) + throws IOException { + Path rootDir = FSUtils.getRootDir(c); + Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); + List logFiles = new ArrayList(); + + FileSystem fs = FileSystem.get(c); + logFiles = getFiles(fs, logDir, logFiles, filter); + logFiles = getFiles(fs, oldLogDir, logFiles, filter); + return logFiles; + } + + /** + * Get list of all old WAL files (WALs and archive) + * @param c - configuration + * @return list of WAL files + * @throws IOException exception + */ + public static List getWALFilesOlderThan(final Configuration c, + final HashMap hostTimestampMap) throws IOException { + Path rootDir = FSUtils.getRootDir(c); + Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); + List logFiles = new ArrayList(); + + PathFilter filter = new PathFilter() { + + @Override + public boolean accept(Path p) { + try { + if (DefaultWALProvider.isMetaFile(p)) { + return false; + } + String host = BackupUtil.parseHostNameFromLogFile(p); + String oldTimestamp = hostTimestampMap.get(host); + String currentLogTS = getCreationTime(p, c); + if (LOG.isDebugEnabled()) { + LOG.debug("path=" + p); + LOG.debug("oldTimestamp=" + oldTimestamp); + LOG.debug("currentLogTS=" + currentLogTS); + } + return Long.parseLong(currentLogTS) <= Long.parseLong(oldTimestamp); + } catch (IOException e) { + LOG.error(e); + return false; + } + } + }; + FileSystem fs = FileSystem.get(c); + logFiles = getFiles(fs, logDir, logFiles, filter); + logFiles = getFiles(fs, oldLogDir, logFiles, filter); + return logFiles; + } + + private static List getFiles(FileSystem fs, Path rootDir, List files, + PathFilter filter) throws FileNotFoundException, 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()); + LOG.info(lfs.getPath()); + } + } + return files; + } + + public static String concat(Collection col, String separator) { + if (col.size() == 0) { + return ""; + } + StringBuilder sb = new StringBuilder(); + for (String s : col) { + sb.append(s + separator); + } + sb.deleteCharAt(sb.lastIndexOf(";")); + return sb.toString(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java new file mode 100644 index 00000000000..74411daa722 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -0,0 +1,511 @@ +/** + * + * 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.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.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.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.HStore; +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.util.Bytes; + +/** + * View to an on-disk Backup Image FileSytem + * Provides the set of methods necessary to interact with the on-disk Backup Image data. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class HBackupFileSystem { + public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class); + + private final String RESTORE_TMP_PATH = "/tmp/restoreTemp"; + private final String[] ignoreDirs = { "recovered.edits" }; + + private final Configuration conf; + private final FileSystem fs; + private final Path backupRootPath; + private final String backupId; + + /** + * Create a view to the on-disk Backup Image. + * @param conf to use + * @param backupPath to where the backup Image stored + * @param backupId represent backup Image + */ + HBackupFileSystem(final Configuration conf, final Path backupRootPath, final String backupId) + throws IOException { + this.conf = conf; + this.fs = backupRootPath.getFileSystem(conf); + this.backupRootPath = backupRootPath; + this.backupId = backupId; // the backup ID for the lead backup Image + } + + /** + * @param tableName is the table backuped + * @return {@link HTableDescriptor} saved in backup image of the table + */ + protected HTableDescriptor getTableDesc(String tableName) throws FileNotFoundException, + IOException { + + Path tableInfoPath = this.getTableInfoPath(tableName); + LOG.debug("tableInfoPath = " + tableInfoPath.toString()); + SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath); + LOG.debug("desc = " + desc.getName()); + SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc); + HTableDescriptor tableDescriptor = manifest.getTableDescriptor(); + /* + * for HBase 0.96 or 0.98 HTableDescriptor tableDescriptor = + * FSTableDescriptors.getTableDescriptorFromFs(fs, tableInfoPath); + */ + if (!tableDescriptor.getNameAsString().equals(tableName)) { + LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: " + + tableInfoPath.toString()); + LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString()); + } + return tableDescriptor; + } + + /** + * 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/default/t1_dn/backup_1396650096738" + * @param backupRootDir backup root directory + * @param backupId backup id + * @param table table name + * @return backupPath String for the particular table + */ + protected static String getTableBackupDir(String backupRootDir, String backupId, String table) { + TableName tableName = TableName.valueOf(table); + return backupRootDir + File.separator + tableName.getNamespaceAsString() + File.separator + + tableName.getQualifierAsString() + File.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/backup1/default/t1_dn/backup_1396650096738" + * @param tableN table name + * @return backupPath for the particular table + */ + protected Path getTableBackupPath(String tableN) { + TableName tableName = TableName.valueOf(tableN); + return new Path(this.backupRootPath, tableName.getNamespaceAsString() + File.separator + + tableName.getQualifierAsString() + File.separator + backupId); + } + + /** + * return value represent path for: + * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/.hbase-snapshot" + * @param tableName table name + * @return path for snapshot + */ + protected Path getTableSnapshotPath(String tableName) { + return new Path(this.getTableBackupPath(tableName), HConstants.SNAPSHOT_DIR_NAME); + } + + /** + * return value represent path for: + * "..../default/t1_dn/backup_1396650096738/.hbase-snapshot/snapshot_1396650097621_default_t1_dn" + * 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 FileNotFoundException exception + * @throws IOException exception + */ + protected Path getTableInfoPath(String tableName) throws FileNotFoundException, IOException { + + Path tableSnapShotPath = this.getTableSnapshotPath(tableName); + Path tableInfoPath = null; + + // can't build the path directly as the timestamp values are different + FileStatus[] snapshots = fs.listStatus(tableSnapShotPath); + for (FileStatus snapshot : snapshots) { + tableInfoPath = snapshot.getPath(); + // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest"; + if (tableInfoPath.getName().endsWith("data.manifest")) { + LOG.debug("find Snapshot Manifest"); + break; + } + } + return tableInfoPath; + } + + /** + * return value represent path for: + * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn" + * @param tabelName table name + * @return path to table archive + * @throws IOException exception + */ + protected Path getTableArchivePath(String tableName) throws IOException { + Path baseDir = new Path(getTableBackupPath(tableName), HConstants.HFILE_ARCHIVE_DIRECTORY); + Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR); + Path archivePath = new Path(dataDir, TableName.valueOf(tableName).getNamespaceAsString()); + Path tableArchivePath = + new Path(archivePath, TableName.valueOf(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; + } + + /** + * 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" + */ + protected static String getLogBackupDir(String backupRootDir, String backupId) { + return backupRootDir + File.separator + HConstants.HREGION_LOGDIR_NAME + File.separator + + backupId; + } + + protected static Path getLogBackupPath(String backupRootDir, String backupId) { + return new Path(getLogBackupDir(backupRootDir, backupId)); + } + + private Path getManifestPath(String tableName) throws IOException { + Path manifestPath = new Path(getTableBackupPath(tableName), BackupManifest.FILE_NAME); + + LOG.debug("Looking for " + manifestPath.toString()); + if (!fs.exists(manifestPath)) { + // check log dir for incremental backup case + manifestPath = + new Path(getLogBackupDir(this.backupRootPath.toString(), this.backupId) + File.separator + + BackupManifest.FILE_NAME); + LOG.debug("Looking for " + manifestPath.toString()); + if (!fs.exists(manifestPath)) { + String errorMsg = + "Could not find backup manifest for " + backupId + " in " + backupRootPath.toString(); + throw new IOException(errorMsg); + } + } + return manifestPath; + } + + protected BackupManifest getManifest(String tableName) throws IOException { + BackupManifest manifest = new BackupManifest(conf, this.getManifestPath(tableName)); + return manifest; + } + + /** + * Gets region list + * @param tableName table name + * @return RegionList region list + * @throws FileNotFoundException exception + * @throws IOException exception + */ + + protected ArrayList getRegionList(String tableName) throws FileNotFoundException, + IOException { + Path tableArchivePath = this.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; + } + + /** + * Gets region list + * @param tableArchivePath table archive path + * @return RegionList region list + * @throws FileNotFoundException exception + * @throws IOException exception + */ + protected ArrayList getRegionList(Path tableArchivePath) throws FileNotFoundException, + 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; + } + + /** + * Counts the number of files in all subdirectories of an HBase tables, i.e. HFiles. And finds the + * maximum number of files in one HBase table. + * @param tableArchivePath archive path + * @return the maximum number of files found in 1 HBase table + * @throws IOException exception + */ + protected int getMaxNumberOfFilesInSubDir(Path tableArchivePath) throws IOException { + int result = 1; + ArrayList regionPathList = this.getRegionList(tableArchivePath); + // tableArchivePath = this.getTableArchivePath(tableName); + + if (regionPathList == null || regionPathList.size() == 0) { + throw new IllegalStateException("Cannot restore hbase table because directory '" + + tableArchivePath + "' is not a directory."); + } + + for (Path regionPath : regionPathList) { + result = Math.max(result, getNumberOfFilesInDir(regionPath)); + } + return result; + } + + /** + * Counts the number of files in all subdirectories of an HBase table, i.e. HFiles. + * @param regionPath Path to an HBase table directory + * @return the number of files all directories + * @throws IOException exception + */ + protected int getNumberOfFilesInDir(Path regionPath) throws IOException { + int result = 0; + + if (!fs.exists(regionPath) || !fs.getFileStatus(regionPath).isDirectory()) { + throw new IllegalStateException("Cannot restore hbase table because directory '" + + regionPath.toString() + "' is not a directory."); + } + + FileStatus[] tableDirContent = fs.listStatus(regionPath); + for (FileStatus subDirStatus : tableDirContent) { + FileStatus[] colFamilies = fs.listStatus(subDirStatus.getPath()); + for (FileStatus colFamilyStatus : colFamilies) { + FileStatus[] colFamilyContent = fs.listStatus(colFamilyStatus.getPath()); + result += colFamilyContent.length; + } + } + return result; + } + + /** + * Duplicate the backup image if it's on local cluster + * @see HStore#bulkLoadHFile(String, long) + * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum) + * @param tableArchivePath archive path + * @return the new tableArchivePath + * @throws IOException exception + */ + protected Path checkLocalAndBackup(Path tableArchivePath) throws IOException { + // Move the file if it's on local cluster + boolean isCopyNeeded = false; + + FileSystem srcFs = tableArchivePath.getFileSystem(conf); + FileSystem desFs = FileSystem.get(conf); + if (tableArchivePath.getName().startsWith("/")) { + isCopyNeeded = true; + } else { + // This should match what is done in @see HRegionFileSystem#bulkLoadStoreFile(String, Path, + // long) + if (srcFs.getUri().equals(desFs.getUri())) { + LOG.debug("cluster hold the backup image: " + srcFs.getUri() + "; local cluster node: " + + desFs.getUri()); + isCopyNeeded = true; + } + } + if (isCopyNeeded) { + LOG.debug("File " + tableArchivePath + " on local cluster, back it up before restore"); + Path tmpPath = new Path(RESTORE_TMP_PATH); + if (desFs.exists(tmpPath)) { + try { + desFs.delete(tmpPath, true); + } catch (IOException e) { + LOG.debug("Failed to delete path: " + tmpPath + + ", need to check whether restore target DFS cluster is healthy"); + } + } + FileUtil.copy(srcFs, tableArchivePath, desFs, tmpPath, false, conf); + LOG.debug("Copied to temporary path on local cluster: " + tmpPath); + tableArchivePath = tmpPath; + } + return tableArchivePath; + } + + /** + * 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 + */ + protected byte[][] generateBoundaryKeys(ArrayList regionDirList) + throws FileNotFoundException, IOException { + TreeMap map = new TreeMap(Bytes.BYTES_COMPARATOR); + // Build a set of keys to store the boundaries + byte[][] keys = null; + // 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, new CacheConfig(conf), conf); + final byte[] first, last; + try { + reader.loadFileInfo(); + first = reader.getFirstRowKey(); + last = reader.getLastRowKey(); + 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(); + } + } + } + } + keys = LoadIncrementalHFiles.inferBoundaries(map); + return keys; + } + + /** + * Check whether the backup path exist + * @param backupStr backup + * @param conf configuration + * @return Yes if path exists + * @throws IOException exception + */ + protected 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(); + LOG.debug("Schema of given url: " + backupStr + " is: " + targetFsScheme); + if (fileSys.exists(backupPath)) { + isExist = true; + } + return isExist; + } + + /** + * 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 + */ + protected void checkImageManifestExist(HashMap backupManifestMap, + String[] tableArray) throws IOException { + + try { + for (String tableName : tableArray) { + BackupManifest manifest = this.getManifest(tableName); + backupManifestMap.put(tableName, manifest); + } + } catch (IOException e) { + String expMsg = e.getMessage(); + if (expMsg.contains("No FileSystem for scheme")) { + if (expMsg.contains("gpfs")) { + LOG.error("Please change to use webhdfs url when " + + "the backup image to restore locates on gpfs cluster"); + } else { + LOG.error("Unsupported filesystem scheme found in the backup target url, " + + "please check the url to make sure no typo in it"); + } + throw e; + } else if (expMsg.contains("no authority supported")) { + LOG.error("Please change to use webhdfs url when " + + "the backup image to restore locates on gpfs cluster"); + throw e; + } else { + LOG.error(expMsg); + throw e; + } + } + } + + public static String join(String[] names) { + StringBuilder sb = new StringBuilder(); + String sep = BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND; + for (String s : names) { + sb.append(sep).append(s); + } + return sb.toString(); + } + +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java new file mode 100644 index 00000000000..e91857f47b2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java @@ -0,0 +1,269 @@ +/** + * 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.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; + +/** + * 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.Public +@InterfaceStability.Evolving +public class IncrementalBackupManager { + // parent manager + private BackupManager backupManager; + + public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class); + + public IncrementalBackupManager(BackupManager bm) { + this.backupManager = bm; + } + + /** + * Obtain the list of logs that need to be copied out for this incremental backup. The list is set + * in BackupContext. + * @param backupContext backup context + * @return The new HashMap of RS log timestamps after the log roll for this incremental backup. + * @throws IOException exception + */ + public HashMap getIncrBackupLogFileList(BackupContext backupContext) + throws IOException { + List logList; + HashMap newTimestamps; + HashMap previousTimestampMins; + + Configuration conf = BackupUtil.getConf(); + String savedStartCode = backupManager.readBackupStartCode(); + + // key: tableName + // value: + HashMap> previousTimestampMap = + backupManager.readLogTimestampMap(); + + previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap); + + LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId()); + LOG.debug("Timestamps " + previousTimestampMap); + // 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 hbase:backup. " + + "In order to create an incremental backup, at least one full backup is needed."); + } + + HBaseAdmin hbadmin = null; + Connection conn = null; + try { + LOG.info("Execute roll log procedure for incremental backup ..."); + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap()); + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if(conn != null){ + conn.close(); + } + } + + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + + logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode); + + backupContext.setIncrBackupFileList(logList); + + return newTimestamps; + } + + /** + * 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(HashMap olderTimestamps, + HashMap newestTimestamps, Configuration conf, String savedStartCode) + throws IOException { + LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps + + "\n newestTimestamps: " + newestTimestamps); + Path rootdir = FSUtils.getRootDir(conf); + Path logDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME); + Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME); + FileSystem fs = rootdir.getFileSystem(conf); + NewestLogFilter pathFilter = new NewestLogFilter(conf); + + List resultLogFiles = new ArrayList(); + List newestLogs = new ArrayList(); + + /* + * The old region servers and timestamps info we kept in hbase:backup 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 hbase:backup 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 hbase:backup at the end of successful backup. + */ + + FileStatus[] rss; + Path p; + String host; + String oldTimeStamp; + String currentLogFile; + String currentLogTS; + + // Get the files in .logs. + rss = fs.listStatus(logDir); + for (FileStatus rs : rss) { + p = rs.getPath(); + host = DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname(); + FileStatus[] logs; + oldTimeStamp = olderTimestamps.get(host); + // It is possible that there is no old timestamp in hbase:backup 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 (DefaultWALProvider.isMetaFile(log.getPath())) { + LOG.debug("Skip hbase:meta log file: " + log.getPath().getName()); + continue; + } + currentLogFile = log.getPath().toString(); + resultLogFiles.add(currentLogFile); + currentLogTS = BackupUtil.getCreationTime(log.getPath(), conf); + // newestTimestamps is up-to-date with the current list of hosts + // so newestTimestamps.get(host) will not be null. + if (Long.valueOf(currentLogTS) > Long.valueOf(newestTimestamps.get(host))) { + newestLogs.add(currentLogFile); + } + } + } + + // Include the .oldlogs files too. + FileStatus[] oldlogs = fs.listStatus(oldLogDir); + for (FileStatus oldlog : oldlogs) { + p = oldlog.getPath(); + currentLogFile = p.toString(); + if (DefaultWALProvider.isMetaFile(p)) { + LOG.debug("Skip .meta log file: " + currentLogFile); + continue; + } + host = BackupUtil.parseHostFromOldLog(p); + currentLogTS = BackupUtil.getCreationTime(p, conf); + oldTimeStamp = olderTimestamps.get(host); + /* + * It is possible that there is no old timestamp in hbase:backup 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 (Long.valueOf(currentLogTS) < Long.valueOf(savedStartCode)) { + // This log file is really old, its region server was before our last backup. + continue; + } else { + resultLogFiles.add(currentLogFile); + } + } else if (Long.valueOf(currentLogTS) > Long.valueOf(oldTimeStamp)) { + resultLogFiles.add(currentLogFile); + } + + LOG.debug("resultLogFiles before removal of newestLogs: " + resultLogFiles); + // 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. + String newTimestamp = newestTimestamps.get(host); + if (newTimestamp != null && Long.valueOf(currentLogTS) > Long.valueOf(newTimestamp)) { + newestLogs.add(currentLogFile); + } + } + LOG.debug("newestLogs: " + newestLogs); + // remove newest log per host because they are still in use + resultLogFiles.removeAll(newestLogs); + LOG.debug("resultLogFiles after removal of newestLogs: " + resultLogFiles); + return resultLogFiles; + } + + class NewestLogFilter implements PathFilter { + private String lastBackupTS = "0"; + final private Configuration conf; + + public NewestLogFilter(Configuration conf) { + this.conf = conf; + } + + protected void setLastBackupTS(String ts) { + this.lastBackupTS = ts; + } + + @Override + public boolean accept(Path path) { + // skip meta table log -- ts.meta file + if (DefaultWALProvider.isMetaFile(path)) { + LOG.debug("Skip .meta log file: " + path.getName()); + return false; + } + String timestamp; + try { + timestamp = BackupUtil.getCreationTime(path, conf); + return Long.valueOf(timestamp) > Long.valueOf(lastBackupTS); + } catch (IOException e) { + LOG.warn("Cannot read timestamp of log file " + path); + return false; + } + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java new file mode 100644 index 00000000000..72e4879d39b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface IncrementalRestoreService extends Configurable{ + + public void run(String logDirectory, String[] fromTables, String[] toTables) + throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java new file mode 100644 index 00000000000..ae21b33844c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java @@ -0,0 +1,496 @@ +/** + * 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.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +/** + * The main class which interprets the given arguments and trigger restore operation. + */ + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class RestoreClient { + + private static final Log LOG = LogFactory.getLog(RestoreClient.class); + + private static Options opt; + private static Configuration conf; + private static Set lastRestoreImagesSet; + + // delimiter in tablename list in restore command + private static final String DELIMITER_IN_COMMAND = ","; + + private static final String OPTION_OVERWRITE = "overwrite"; + private static final String OPTION_CHECK = "check"; + private static final String OPTION_AUTOMATIC = "automatic"; + + private static final String USAGE = + "Usage: hbase restore [tableMapping] \n" + + " [-overwrite] [-check] [-automatic]\n" + + " backup_root_path The parent location where the backup images are stored\n" + + " backup_id The id identifying the backup image\n" + + " table(s) Table(s) from the backup image to be restored.\n" + + " Tables are separated by comma.\n" + + " Options:\n" + + " tableMapping A comma separated list of target tables.\n" + + " If specified, each table in must have a mapping.\n" + + " -overwrite With this option, restore overwrites to the existing table " + + "if there's any in\n" + + " restore target. The existing table must be online before restore.\n" + + " -check With this option, restore sequence and dependencies are checked\n" + + " and verified without executing the restore\n" + + " -automatic With this option, all the dependencies are automatically restored\n" + + " together with this backup image following the correct order.\n" + + " The restore dependencies can be checked by using \"-check\" " + + "option,\n" + + " or using \"hbase backup describe\" command. Without this option, " + + "only\n" + " this backup image is restored\n"; + + private RestoreClient(){ + throw new AssertionError("Instantiating utility class..."); + } + + protected static void init() throws IOException { + // define supported options + opt = new Options(); + opt.addOption(OPTION_OVERWRITE, false, + "Overwrite the data if any of the restore target tables exists"); + opt.addOption(OPTION_CHECK, false, "Check restore sequence and dependencies"); + opt.addOption(OPTION_AUTOMATIC, false, "Restore all dependencies"); + opt.addOption("debug", false, "Enable debug logging"); + + conf = getConf(); + + // disable irrelevant loggers to avoid it mess up command output + disableUselessLoggers(); + } + + public static void main(String[] args) throws IOException { + init(); + parseAndRun(args); + } + + private static void parseAndRun(String[] args) { + CommandLine cmd = null; + try { + cmd = new PosixParser().parse(opt, args); + } catch (ParseException e) { + LOG.error("Could not parse command", e); + System.exit(-1); + } + + // enable debug logging + Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); + if (cmd.hasOption("debug")) { + backupClientLogger.setLevel(Level.DEBUG); + } + + // whether to overwrite to existing table if any, false by default + boolean isOverwrite = cmd.hasOption(OPTION_OVERWRITE); + if (isOverwrite) { + 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"); + } + + // whether to restore all dependencies, false by default + boolean autoRestore = cmd.hasOption(OPTION_AUTOMATIC); + if (autoRestore) { + LOG.debug("Found -automatic option in restore command, " + + "will automatically retore all the dependencies"); + } + + // parse main restore command options + String[] remainArgs = cmd.getArgs(); + if (remainArgs.length < 3) { + System.out.println("ERROR: missing arguments"); + System.out.println(USAGE); + System.exit(-1); + } + + String backupRootDir = remainArgs[0]; + String backupId = remainArgs[1]; + String tables = remainArgs[2]; + + String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null; + + String[] sTableArray = (tables != null) ? tables.split(DELIMITER_IN_COMMAND) : null; + String[] tTableArray = (tableMapping != null) ? tableMapping.split(DELIMITER_IN_COMMAND) : null; + + if (tableMapping != null && tTableArray != null && (sTableArray.length != tTableArray.length)) { + System.err.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping); + System.out.println(USAGE); + System.exit(-1); + } + + try { + HBackupFileSystem hBackupFS = new HBackupFileSystem(conf, new Path(backupRootDir), backupId); + restore_stage1(hBackupFS, backupRootDir, backupId, check, autoRestore, sTableArray, + tTableArray, isOverwrite); + } catch (IOException e) { + System.err.println("ERROR: " + e.getMessage()); + System.exit(-1); + } + } + + /** + * Restore operation. Stage 1: validate backupManifest, and check target tables + * @param hBackupFS to access the backup image + * @param backupRootDir The root dir for backup image + * @param backupId The backup id for image to be restored + * @param check True if only do dependency check + * @param autoRestore True if automatically restore following the dependency + * @param sTableArray The array of tables to be restored + * @param tTableArray The array of mapping tables to restore to + * @param isOverwrite True then do restore overwrite if target table exists, otherwise fail the + * request if target table exists + * @return True if only do dependency check + * @throws IOException if any failure during restore + */ + public static boolean restore_stage1(HBackupFileSystem hBackupFS, String backupRootDir, + String backupId, boolean check, boolean autoRestore, String[] sTableArray, + String[] tTableArray, boolean isOverwrite) throws IOException { + + HashMap backupManifestMap = new HashMap(); + // check and load backup image manifest for the tables + hBackupFS.checkImageManifestExist(backupManifestMap, sTableArray); + + try { + // Check and validate the backup image and its dependencies + if (check || autoRestore) { + if (validate(backupManifestMap)) { + LOG.info("Checking backup images: ok"); + } else { + String errMsg = "Some dependencies are missing for restore"; + LOG.error(errMsg); + throw new IOException(errMsg); + } + } + + // return true if only for check + if (check) { + return true; + } + + if (tTableArray == null) { + tTableArray = sTableArray; + } + + // check the target tables + checkTargetTables(tTableArray, isOverwrite); + + // start restore process + Set restoreImageSet = + restore_stage2(hBackupFS, backupManifestMap, sTableArray, tTableArray, autoRestore); + + LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!"); + lastRestoreImagesSet = restoreImageSet; + + } catch (IOException e) { + LOG.error("ERROR: restore failed with error: " + e.getMessage()); + throw e; + } + + // not only for check, return false + return false; + } + + /** + * Get last restore image set. The value is globally set for the latest finished restore. + * @return the last restore image set + */ + public static Set getLastRestoreImagesSet() { + return lastRestoreImagesSet; + } + + private static boolean validate(HashMap backupManifestMap) + throws IOException { + boolean isValid = true; + + for (Entry manifestEntry : backupManifestMap.entrySet()) { + + String table = manifestEntry.getKey(); + TreeSet imageSet = new TreeSet(); + + ArrayList depList = manifestEntry.getValue().getDependentListByTable(table); + if (depList != null && !depList.isEmpty()) { + imageSet.addAll(depList); + } + + // todo merge + LOG.debug("merge will be implemented in future jira"); + // BackupUtil.clearMergedImages(table, imageSet, conf); + + LOG.info("Dependent image(s) from old to new:"); + for (BackupImage image : imageSet) { + String imageDir = + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table); + if (!HBackupFileSystem.checkPathExist(imageDir, getConf())) { + LOG.error("ERROR: backup image does not exist: " + imageDir); + isValid = false; + break; + } + // TODO More validation? + LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available"); + } + } + return isValid; + } + + /** + * Validate target Tables + * @param tTableArray: target tables + * @param isOverwrite overwrite existing table + * @throws IOException exception + */ + private static void checkTargetTables(String[] tTableArray, boolean isOverwrite) + throws IOException { + ArrayList existTableList = new ArrayList(); + ArrayList disabledTableList = new ArrayList(); + + // check if the tables already exist + HBaseAdmin admin = null; + Connection conn = null; + try { + conn = ConnectionFactory.createConnection(conf); + admin = (HBaseAdmin) conn.getAdmin(); + for (String tableName : tTableArray) { + if (admin.tableExists(TableName.valueOf(tableName))) { + existTableList.add(tableName); + if (admin.isTableDisabled(TableName.valueOf(tableName))) { + disabledTableList.add(tableName); + } + } else { + LOG.info("HBase table " + tableName + + " does not exist. It will be create during backup process"); + } + } + } finally { + if (admin != null) { + admin.close(); + } + if (conn != null) { + conn.close(); + } + } + + if (existTableList.size() > 0) { + if (!isOverwrite) { + LOG.error("Existing table found in the restore target, please add \"-overwrite\" " + + "option in the command if you mean to restore to these existing tables"); + LOG.info("Existing table list in restore target: " + existTableList); + throw new IOException("Existing table found in target while no \"-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. Stage 2: resolved Backup Image dependency + * @param hBackupFS to access the backup image + * @param backupManifestMap : tableName, Manifest + * @param sTableArray The array of tables to be restored + * @param tTableArray The array of mapping tables to restore to + * @param autoRestore : yes, restore all the backup images on the dependency list + * @return set of BackupImages restored + * @throws IOException exception + */ + private static Set restore_stage2(HBackupFileSystem hBackupFS, + HashMap backupManifestMap, String[] sTableArray, + String[] tTableArray, boolean autoRestore) throws IOException { + TreeSet restoreImageSet = new TreeSet(); + + for (int i = 0; i < sTableArray.length; i++) { + restoreImageSet.clear(); + String table = sTableArray[i]; + BackupManifest manifest = backupManifestMap.get(table); + if (autoRestore) { + // Get the image list of this backup for restore in time order from old + // to new. + TreeSet restoreList = + new TreeSet(manifest.getDependentListByTable(table)); + LOG.debug("need to clear merged Image. to be implemented in future jira"); + + for (BackupImage image : restoreList) { + restoreImage(image, table, tTableArray[i]); + } + restoreImageSet.addAll(restoreList); + } else { + BackupImage image = manifest.getBackupImage(); + List depList = manifest.getDependentListByTable(table); + // The dependency list always contains self. + if (depList != null && depList.size() > 1) { + LOG.warn("Backup image " + image.getBackupId() + " depends on other images.\n" + + "this operation will only restore the delta contained within backupImage " + + image.getBackupId()); + } + restoreImage(image, table, tTableArray[i]); + restoreImageSet.add(image); + } + + if (autoRestore) { + 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)); + } + } + } + + } + return restoreImageSet; + } + + /** + * Restore operation handle each backupImage + * @param image: backupImage + * @param sTable: table to be restored + * @param tTable: table to be restored to + * @throws IOException exception + */ + private static void restoreImage(BackupImage image, String sTable, String tTable) + throws IOException { + + Configuration conf = getConf(); + + String rootDir = image.getRootDir(); + LOG.debug("Image root dir " + rootDir); + String backupId = image.getBackupId(); + + HBackupFileSystem hFS = new HBackupFileSystem(conf, new Path(rootDir), backupId); + RestoreUtil restoreTool = new RestoreUtil(conf, hFS); + BackupManifest manifest = hFS.getManifest(sTable); + + Path tableBackupPath = hFS.getTableBackupPath(sTable); + + // todo: convert feature will be provided in a future jira + boolean converted = false; + + if (manifest.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL) || converted) { + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from " + + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString()); + restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted); + } else { // incremental Backup + String logBackupDir = + HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId()); + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image " + + logBackupDir); + restoreTool.incrementalRestoreTable(logBackupDir, new String[] { sTable }, + new String[] { tTable }); + } + + LOG.info(sTable + " has been successfully restored to " + tTable); + } + + /** + * Set the configuration from a given one. + * @param newConf A new given configuration + */ + public synchronized static void setConf(Configuration newConf) { + conf = newConf; + } + + /** + * Get and merge Hadoop and HBase configuration. + * @throws IOException exception + */ + protected static Configuration getConf() { + if (conf == null) { + synchronized (RestoreClient.class) { + conf = new Configuration(); + HBaseConfiguration.merge(conf, HBaseConfiguration.create()); + } + } + return conf; + } + + private static void disableUselessLoggers() { + // disable zookeeper log to avoid it mess up command output + Logger zkLogger = Logger.getLogger("org.apache.zookeeper"); + LOG.debug("Zookeeper log level before set: " + zkLogger.getLevel()); + zkLogger.setLevel(Level.OFF); + LOG.debug("Zookeeper log level after set: " + zkLogger.getLevel()); + + // disable hbase zookeeper tool log to avoid it mess up command output + Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper"); + LOG.debug("HBase zookeeper log level before set: " + hbaseZkLogger.getLevel()); + hbaseZkLogger.setLevel(Level.OFF); + LOG.debug("HBase Zookeeper log level after set: " + hbaseZkLogger.getLevel()); + + // disable hbase client log to avoid it mess up command output + Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client"); + LOG.debug("HBase client log level before set: " + hbaseClientLogger.getLevel()); + hbaseClientLogger.setLevel(Level.OFF); + LOG.debug("HBase client log level after set: " + hbaseClientLogger.getLevel()); + + // disable other related log to avoid mess up command output + Logger otherLogger = Logger.getLogger("org.apache.hadoop.hbase.io.hfile"); + otherLogger.setLevel(Level.OFF); + otherLogger = Logger.getLogger("org.apache.hadoop.hbase.util"); + otherLogger.setLevel(Level.OFF); + otherLogger = Logger.getLogger("org.apache.hadoop.hbase.mapreduce"); + otherLogger.setLevel(Level.OFF); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java new file mode 100644 index 00000000000..bdb7988b5b7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java @@ -0,0 +1,503 @@ +/** + * 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.EOFException; +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.NavigableSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; + +/** + * A collection for methods used by multiple classes to restore HBase tables. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class RestoreUtil { + + public static final Log LOG = LogFactory.getLog(RestoreUtil.class); + + protected Configuration conf = null; + + protected HBackupFileSystem hBackupFS = null; + + // store table name and snapshot dir mapping + private final HashMap snapshotMap = new HashMap(); + + public RestoreUtil(Configuration conf, HBackupFileSystem hBackupFS) throws IOException { + this.conf = conf; + this.hBackupFS = hBackupFS; + } + + /** + * 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 logDir : incremental backup folders, which contains WAL + * @param tableNames : source tableNames(table names were backuped) + * @param newTableNames : target tableNames(table names to be restored to) + * @throws IOException exception + */ + public void incrementalRestoreTable(String logDir, String[] tableNames, String[] newTableNames) + throws IOException { + + if (tableNames.length != newTableNames.length) { + throw new IOException("Number of source tables adn taget Tables does not match!"); + } + + // for incremental backup image, expect the table already created either by user or previous + // full backup. Here, check that all new tables exists + HBaseAdmin admin = null; + Connection conn = null; + try { + conn = ConnectionFactory.createConnection(conf); + admin = (HBaseAdmin) conn.getAdmin(); + for (String tableName : newTableNames) { + if (!admin.tableExists(TableName.valueOf(tableName))) { + admin.close(); + throw new IOException("HBase table " + tableName + + " does not exist. Create the table first, e.g. by restoring a full backup."); + } + } + IncrementalRestoreService restoreService = + BackupRestoreServiceFactory.getIncrementalRestoreService(conf); + + restoreService.run(logDir, tableNames, newTableNames); + } finally { + if (admin != null) { + admin.close(); + } + if(conn != null){ + conn.close(); + } + } + } + + public void fullRestoreTable(Path tableBackupPath, String tableName, String newTableName, + boolean converted) throws IOException { + + restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted); + } + + private void restoreTableAndCreate(String tableName, String newTableName, Path tableBackupPath, + boolean converted) throws IOException { + if (newTableName == null || newTableName.equals("")) { + newTableName = tableName; + } + + FileSystem fileSys = tableBackupPath.getFileSystem(this.conf); + + // get table descriptor first + HTableDescriptor tableDescriptor = null; + + Path tableSnapshotPath = hBackupFS.getTableSnapshotPath(tableName); + + 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(); + LOG.debug("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString() + + " while tableName = " + tableName); + // HBase 96.0 and 98.0 + // tableDescriptor = + // FSTableDescriptors.getTableDescriptorFromFs(fileSys, snapshotMap.get(tableName)); + } else { + tableDescriptor = hBackupFS.getTableDesc(tableName); + LOG.debug("tableSnapshotPath=" + tableSnapshotPath.toString()); + snapshotMap.put(tableName, hBackupFS.getTableInfoPath(tableName)); + } + if (tableDescriptor == null) { + LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost"); + } + } else if (converted) { + // first check if this is a converted backup image + LOG.error("convert will be supported in a future jira"); + } + + Path tableArchivePath = hBackupFS.getTableArchivePath(tableName); + if (tableArchivePath == null) { + if (tableDescriptor != null) { + // find table descriptor but no archive dir means the table is empty, create table and exit + LOG.debug("find table descriptor but no archive dir for table " + tableName + + ", will only create table"); + tableDescriptor.setName(Bytes.toBytes(newTableName)); + checkAndCreateTable(tableBackupPath, tableName, newTableName, null, tableDescriptor); + return; + } else { + throw new IllegalStateException("Cannot restore hbase table because directory '" + + " tableArchivePath is null."); + } + } + + if (tableDescriptor == null) { + tableDescriptor = new HTableDescriptor(newTableName); + } else { + tableDescriptor.setName(Bytes.toBytes(newTableName)); + } + + if (!converted) { + // record all region dirs: + // load all files in dir + try { + ArrayList regionPathList = hBackupFS.getRegionList(tableName); + + // should only try to create the table with all region informations, so we could pre-split + // the regions in fine grain + checkAndCreateTable(tableBackupPath, tableName, newTableName, regionPathList, + tableDescriptor); + if (tableArchivePath != null) { + // start real restore through bulkload + // if the backup target is on local cluster, special action needed + Path tempTableArchivePath = hBackupFS.checkLocalAndBackup(tableArchivePath); + if (tempTableArchivePath.equals(tableArchivePath)) { + LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath); + } else { + regionPathList = hBackupFS.getRegionList(tempTableArchivePath); // point to the tempDir + LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath); + } + + LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false); + for (Path regionPath : regionPathList) { + String regionName = regionPath.toString(); + LOG.debug("Restoring HFiles from directory " + regionName); + String[] args = { regionName, newTableName }; + loader.run(args); + } + } + // restore the recovered.edits if exists + replayRecoveredEditsIfAny(tableBackupPath, tableName, tableDescriptor); + } catch (Exception e) { + throw new IllegalStateException("Cannot restore hbase table", e); + } + } else { + LOG.debug("convert will be supported in a future jira"); + } + } + + /** + * Replay recovered edits from backup. + */ + private void replayRecoveredEditsIfAny(Path tableBackupPath, String tableName, + HTableDescriptor newTableHtd) throws IOException { + + LOG.debug("Trying to replay the recovered.edits if exist to the target table " + + newTableHtd.getNameAsString() + " from the backup of table " + tableName + "."); + + FileSystem fs = tableBackupPath.getFileSystem(this.conf); + ArrayList regionDirs = hBackupFS.getRegionList(tableName); + + if (regionDirs == null || regionDirs.size() == 0) { + LOG.warn("No recovered.edits to be replayed for empty backup of table " + tableName + "."); + return; + } + + Connection conn = null; + try { + + conn = ConnectionFactory.createConnection(conf); + + for (Path regionDir : regionDirs) { + // OLD: NavigableSet files = HLogUtil.getSplitEditFilesSorted(fs, regionDir); + NavigableSet files = WALSplitter.getSplitEditFilesSorted(fs, regionDir); + + if (files == null || files.isEmpty()) { + LOG.warn("No recovered.edits found for the region " + regionDir.getName() + "."); + return; + } + + for (Path edits : files) { + if (edits == null || !fs.exists(edits)) { + LOG.warn("Null or non-existent edits file: " + edits); + continue; + } + + HTable table = null; + try { + table = (HTable) conn.getTable(newTableHtd.getTableName()); + replayRecoveredEdits(table, fs, edits); + table.flushCommits(); + table.close(); + } catch (IOException e) { + boolean skipErrors = conf.getBoolean("hbase.skip.errors", false); + if (skipErrors) { + Path p = WALSplitter.moveAsideBadEditsFile(fs, edits); + LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + + "=true so continuing. Renamed " + edits + " as " + p, e); + } else { + throw e; + } + } finally { + if (table != null) { + table.close(); + } + } + } // for each edit file under a region + } // for each region + + } finally { + if (conn != null) { + conn.close(); + } + } + } + + /** + * Restore process for an edit entry. + * @param htable The target table of restore + * @param key HLog key + * @param val KVs + * @throws IOException exception + */ + private void restoreEdit(HTable htable, WALKey key, WALEdit val) throws IOException { + Put put = null; + Delete del = null; + Cell lastKV = null; + for (Cell kv : val.getCells()) { + // filtering HLog meta entries, see HLog.completeCacheFlushLogEdit + if (WALEdit.isMetaEditFamily(CellUtil.cloneFamily(kv))) { + continue; + } + + // A WALEdit may contain multiple operations (HBASE-3584) and/or + // multiple rows (HBASE-5229). + // Aggregate as much as possible into a single Put/Delete + // operation before apply the action to the table. + if (lastKV == null || lastKV.getTypeByte() != kv.getTypeByte() + || !CellUtil.matchingRow(lastKV, kv)) { + // row or type changed, write out aggregate KVs. + if (put != null) { + applyAction(htable, put); + } + if (del != null) { + applyAction(htable, del); + } + + if (CellUtil.isDelete(kv)) { + del = new Delete(CellUtil.cloneRow(kv)); + } else { + put = new Put(CellUtil.cloneRow(kv)); + } + } + if (CellUtil.isDelete(kv)) { + del.addDeleteMarker(kv); + } else { + put.add(kv); + } + lastKV = kv; + } + // write residual KVs + if (put != null) { + applyAction(htable, put); + } + if (del != null) { + applyAction(htable, del); + } + } + + /** + * Apply an action (Put/Delete) to table. + * @param table table + * @param action action + * @throws IOException exception + */ + private void applyAction(HTable table, Mutation action) throws IOException { + // The actions are not immutable, so we defensively copy them + if (action instanceof Put) { + Put put = new Put((Put) action); + // put.setWriteToWAL(false); + // why do not we do WAL? + put.setDurability(Durability.SKIP_WAL); + table.put(put); + } else if (action instanceof Delete) { + Delete delete = new Delete((Delete) action); + table.delete(delete); + } else { + throw new IllegalArgumentException("action must be either Delete or Put"); + } + } + + /** + * Replay the given edits. + * @param htable The target table of restore + * @param fs File system + * @param edits Recovered.edits to be replayed + * @throws IOException exception + */ + private void replayRecoveredEdits(HTable htable, FileSystem fs, Path edits) throws IOException { + LOG.debug("Replaying edits from " + edits + "; path=" + edits); + + WAL.Reader reader = null; + try { + reader = WALFactory.createReader(fs, edits, this.conf); + long editsCount = 0; + WAL.Entry entry; + + try { + while ((entry = reader.next()) != null) { + restoreEdit(htable, entry.getKey(), entry.getEdit()); + editsCount++; + } + LOG.debug(editsCount + " edits from " + edits + " have been replayed."); + + } catch (EOFException eof) { + Path p = WALSplitter.moveAsideBadEditsFile(fs, edits); + String msg = + "Encountered EOF. Most likely due to Master failure during " + + "log spliting, so we have this data in another edit. " + + "Continuing, but renaming " + edits + " as " + p; + LOG.warn(msg, eof); + } catch (IOException ioe) { + // If the IOE resulted from bad file format, + // then this problem is idempotent and retrying won't help + if (ioe.getCause() instanceof ParseException) { + Path p = WALSplitter.moveAsideBadEditsFile(fs, edits); + String msg = + "File corruption encountered! " + "Continuing, but renaming " + edits + " as " + p; + LOG.warn(msg, ioe); + } else { + // other IO errors may be transient (bad network connection, + // checksum exception on one datanode, etc). throw & retry + throw ioe; + } + } + } finally { + if (reader != null) { + reader.close(); + } + } + } + + /** + * Create a {@link LoadIncrementalHFiles} instance to be used to restore the HFiles of a full + * backup. + * @return the {@link LoadIncrementalHFiles} instance + * @throws IOException exception + */ + private LoadIncrementalHFiles createLoader(Path tableArchivePath, boolean multipleTables) + throws IOException { + // set configuration for restore: + // LoadIncrementalHFile needs more time + // hbase.rpc.timeout 600000 + // calculates + Integer milliSecInMin = 60000; + Integer previousMillis = this.conf.getInt("hbase.rpc.timeout", 0); + Integer numberOfFilesInDir = + multipleTables ? hBackupFS.getMaxNumberOfFilesInSubDir(tableArchivePath) : hBackupFS + .getNumberOfFilesInDir(tableArchivePath); + Integer calculatedMillis = numberOfFilesInDir * milliSecInMin; // 1 minute per file + Integer resultMillis = Math.max(calculatedMillis, previousMillis); + if (resultMillis > previousMillis) { + LOG.info("Setting configuration for restore with LoadIncrementalHFile: " + + "hbase.rpc.timeout to " + calculatedMillis / milliSecInMin + + " minutes, to handle the number of files in backup " + tableArchivePath); + this.conf.setInt("hbase.rpc.timeout", resultMillis); + } + + LoadIncrementalHFiles loader = null; + try { + loader = new LoadIncrementalHFiles(this.conf); + } catch (Exception e1) { + throw new IOException(e1); + } + return loader; + } + + /** + * Prepare the table for bulkload, most codes copied from + * {@link LoadIncrementalHFiles#createTable(String, String)} + * @param tableBackupPath path + * @param tableName table name + * @param targetTableName target table name + * @param regionDirList region directory list + * @param htd table descriptor + * @throws IOException exception + */ + private void checkAndCreateTable(Path tableBackupPath, String tableName, String targetTableName, + ArrayList regionDirList, HTableDescriptor htd) throws IOException { + HBaseAdmin hbadmin = null; + Connection conn = null; + try { + conn = ConnectionFactory.createConnection(conf); + hbadmin = (HBaseAdmin) conn.getAdmin(); + if (hbadmin.tableExists(TableName.valueOf(targetTableName))) { + LOG.info("Using exising target table '" + targetTableName + "'"); + } else { + LOG.info("Creating target table '" + targetTableName + "'"); + + // if no region dir given, create the table and return + if (regionDirList == null || regionDirList.size() == 0) { + + hbadmin.createTable(htd); + return; + } + + byte[][] keys = hBackupFS.generateBoundaryKeys(regionDirList); + + // create table using table decriptor and region boundaries + hbadmin.createTable(htd, keys); + } + } catch (Exception e) { + throw new IOException(e); + } finally { + if (hbadmin != null) { + hbadmin.close(); + } + if(conn != null){ + conn.close(); + } + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java new file mode 100644 index 00000000000..a3b5db5dda5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java @@ -0,0 +1,292 @@ +/** + * 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.Method; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.backup.BackupCopyService; +import org.apache.hadoop.hbase.backup.BackupHandler; +import org.apache.hadoop.hbase.backup.BackupUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.snapshot.ExportSnapshot; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.tools.DistCp; +import org.apache.hadoop.tools.DistCpConstants; +import org.apache.hadoop.tools.DistCpOptions; +/** + * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot, + * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper + * implementation. The other is copying for incremental log files, which bases on extending + * DistCp's function with copy progress reporting to ZooKeeper implementation. + * + * For now this is only a wrapper. The other features such as progress and increment backup will be + * implemented in future jira + */ + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class MapReduceBackupCopyService implements BackupCopyService { + private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyService.class); + + private Configuration conf; + // private static final long BYTES_PER_MAP = 2 * 256 * 1024 * 1024; + + // 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 MapReduceBackupCopyService() { + } + + @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(BackupHandler, Configuration, Type, String[])} + * @param subTaskPercntgInWholeTask The percentage of the copy subtask + */ + public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) { + this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask; + } + + class SnapshotCopy extends ExportSnapshot { + private BackupHandler backupHandler; + private String table; + + public SnapshotCopy(BackupHandler backupHandler, String table) { + super(); + this.backupHandler = backupHandler; + this.table = table; + } + + public BackupHandler getBackupHandler() { + return this.backupHandler; + } + + public String getTable() { + return this.table; + } + } + + // Extends DistCp for progress updating to hbase:backup + // 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 BackupHandler backupHandler; + + public BackupDistCp(Configuration conf, DistCpOptions options, BackupHandler backupHandler) + throws Exception { + super(conf, options); + this.backupHandler = backupHandler; + } + + @Override + public Job execute() throws Exception { + + // reflection preparation for private methods and fields + Class classDistCp = org.apache.hadoop.tools.DistCp.class; + Method methodCreateMetaFolderPath = classDistCp.getDeclaredMethod("createMetaFolderPath"); + Method methodCreateJob = classDistCp.getDeclaredMethod("createJob"); + Method methodCreateInputFileListing = + classDistCp.getDeclaredMethod("createInputFileListing", Job.class); + Method methodCleanup = classDistCp.getDeclaredMethod("cleanup"); + + Field fieldInputOptions = classDistCp.getDeclaredField("inputOptions"); + Field fieldMetaFolder = classDistCp.getDeclaredField("metaFolder"); + Field fieldJobFS = classDistCp.getDeclaredField("jobFS"); + Field fieldSubmitted = classDistCp.getDeclaredField("submitted"); + + methodCreateMetaFolderPath.setAccessible(true); + methodCreateJob.setAccessible(true); + methodCreateInputFileListing.setAccessible(true); + methodCleanup.setAccessible(true); + + fieldInputOptions.setAccessible(true); + fieldMetaFolder.setAccessible(true); + fieldJobFS.setAccessible(true); + fieldSubmitted.setAccessible(true); + + // execute() logic starts here + assert fieldInputOptions.get(this) != null; + assert getConf() != null; + + Job job = null; + try { + synchronized (this) { + // Don't cleanup while we are setting up. + fieldMetaFolder.set(this, methodCreateMetaFolderPath.invoke(this)); + fieldJobFS.set(this, ((Path) fieldMetaFolder.get(this)).getFileSystem(getConf())); + + job = (Job) methodCreateJob.invoke(this); + } + methodCreateInputFileListing.invoke(this, job); + + // Get the total length of the source files + List srcs = ((DistCpOptions) fieldInputOptions.get(this)).getSourcePaths(); + long totalSrcLgth = 0; + for (Path aSrc : srcs) { + totalSrcLgth += BackupUtil.getFilesLength(aSrc.getFileSystem(getConf()), aSrc); + } + + // submit the copy job + job.submit(); + fieldSubmitted.set(this, true); + + // after submit the MR job, set its handler in backup handler for cancel process + // this.backupHandler.copyJob = job; + + // Update the copy progress to ZK every 0.5s if progress value changed + int progressReportFreq = + 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); + this.backupHandler.updateProgress(newProgressStr, bytesCopied); + LOG.debug("Backup progress data updated to hbase:backup: \"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); + + // accumulate the overall backup progress + progressDone = newProgress; + bytesCopied += totalSrcLgth; + + this.backupHandler.updateProgress(newProgressStr, bytesCopied); + LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr + + " - " + bytesCopied + " bytes copied.\""); + + } finally { + if (!fieldSubmitted.getBoolean(this)) { + methodCleanup.invoke(this); + } + } + + String jobID = job.getJobID().toString(); + job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID); + + LOG.debug("DistCp job-id: " + jobID); + return job; + } + + } + + /** + * Do backup copy based on different types. + * @param handler The backup handler reference + * @param conf The hadoop configuration + * @param copyType The backup copy type + * @param options Options for customized ExportSnapshot or DistCp + * @throws Exception exception + */ + public int copy(BackupHandler handler, Configuration conf, BackupCopyService.Type copyType, + String[] options) throws IOException { + + int res = 0; + + try { + if (copyType == Type.FULL) { + SnapshotCopy snapshotCp = + new SnapshotCopy(handler, handler.getBackupContext().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 == Type.INCREMENTAL) { + LOG.debug("Doing COPY_TYPE_DISTCP"); + setSubTaskPercntgInWholeTask(1f); + + BackupDistCp distcp = new BackupDistCp(new Configuration(conf), null, handler); + // 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)); + if (options.length == 2) { + Path dest = new Path(options[1]); + FileSystem destfs = dest.getFileSystem(conf); + if (!destfs.exists(dest)) { + destfs.mkdirs(dest); + } + } + + res = distcp.run(options); + } + return res; + + } catch (Exception e) { + throw new IOException(e); + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java new file mode 100644 index 00000000000..deefbf7d90b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java @@ -0,0 +1,72 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.IncrementalRestoreService; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class MapReduceRestoreService implements IncrementalRestoreService { + public static final Log LOG = LogFactory.getLog(MapReduceRestoreService.class); + + private WALPlayer player; + + public MapReduceRestoreService() { + this.player = new WALPlayer(); + } + + @Override + public void run(String logDir, String[] tableNames, String[] newTableNames) throws IOException { + String tableStr = HBackupFileSystem.join(tableNames); + String newTableStr = HBackupFileSystem.join(newTableNames); + + // WALPlayer reads all files in arbitrary directory structure and creates a Map task for each + // log file + + String[] playerArgs = { logDir, tableStr, newTableStr }; + LOG.info("Restore incremental backup from directory " + logDir + " from hbase tables " + + HBackupFileSystem.join(tableNames) + " to tables " + + HBackupFileSystem.join(newTableNames)); + try { + player.run(playerArgs); + } catch (Exception e) { + throw new IOException("cannot restore from backup directory " + logDir + + " (check Hadoop and HBase logs) " + e); + } + } + + @Override + public Configuration getConf() { + return player.getConf(); + } + + @Override + public void setConf(Configuration conf) { + this.player.setConf(conf); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java new file mode 100644 index 00000000000..4712548912a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java @@ -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 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 com.google.common.base.Predicate; +import com.google.common.collect.Iterables; + +import java.io.IOException; +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.backup.BackupSystemTable; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate; + + + + +/** + * Implementation of a log cleaner that checks if a log is still scheduled for + * incremental backup before deleting it when its TTL is over. + */ +@InterfaceStability.Evolving +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class BackupLogCleaner extends BaseLogCleanerDelegate { + private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class); + + private boolean stopped = false; + + public BackupLogCleaner() { + } + + @Override + public Iterable getDeletableFiles(Iterable files) { + // all members of this class are null if backup is disabled, + // so we cannot filter the files + if (this.getConf() == null) { + return files; + } + + try { + final BackupSystemTable table = BackupSystemTable.getTable(getConf()); + // If we do not have recorded backup sessions + if (table.hasBackupSessions() == false) { + return files; + } + return Iterables.filter(files, new Predicate() { + @Override + public boolean apply(FileStatus file) { + try { + String wal = file.getPath().toString(); + boolean logInSystemTable = table.checkWALFile(wal); + if (LOG.isDebugEnabled()) { + if (logInSystemTable) { + LOG.debug("Found log file in hbase:backup, deleting: " + wal); + } else { + LOG.debug("Didn't find this log in hbase:backup, keeping: " + wal); + } + } + return logInSystemTable; + } catch (IOException e) { + LOG.error(e); + return false;// keep file for a while, HBase failed + } + } + }); + } catch (IOException e) { + LOG.error("Failed to get hbase:backup table, therefore will keep all files", e); + // nothing to delete + return new ArrayList(); + } + + } + + @Override + public void setConf(Configuration config) { + // If backup is disabled, keep all members null + if (!config.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) { + LOG.warn("Backup is disabled - allowing all wals to be deleted"); + return; + } + super.setConf(config); + } + + @Override + public void stop(String why) { + if (this.stopped) { + return; + } + this.stopped = true; + LOG.info("Stopping BackupLogCleaner"); + } + + @Override + public boolean isStopped() { + return this.stopped; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java new file mode 100644 index 00000000000..f96682fdb85 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.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.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadPoolExecutor; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; +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.ProcedureCoordinator; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription; +import org.apache.zookeeper.KeeperException; + +public class LogRollMasterProcedureManager extends MasterProcedureManager { + + public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc"; + public static final String ROLLLOG_PROCEDURE_NAME = "rolllog"; + private static final Log LOG = LogFactory.getLog(LogRollMasterProcedureManager.class); + + 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 KeeperException, IOException, UnsupportedOperationException { + this.master = master; + this.done = false; + + // setup the default procedure coordinator + String name = master.getServerName().toString(); + ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1); + BaseCoordinatedStateManager coordManager = + (BaseCoordinatedStateManager) CoordinatedStateManagerFactory + .getCoordinatedStateManager(master.getConfiguration()); + coordManager.initialize(master); + + ProcedureCoordinatorRpcs comms = + coordManager.getProcedureCoordinatorRpcs(getProcedureSignature(), name); + + this.coordinator = new ProcedureCoordinator(comms, tpool); + } + + @Override + public String getProcedureSignature() { + return ROLLLOG_PROCEDURE_SIGNATURE; + } + + @Override + public void execProcedure(ProcedureDescription desc) throws IOException { + 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()); + } + Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), new byte[0], 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 boolean isProcedureDone(ProcedureDescription desc) throws IOException { + return done; + } + +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java new file mode 100644 index 00000000000..618748e072f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java @@ -0,0 +1,138 @@ +/** + * 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.util.HashMap; +import java.util.concurrent.Callable; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.backup.BackupSystemTable; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +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.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.wal.FSHLog; + + +/** + * This backup subprocedure implementation forces a log roll on the RS. + */ +public class LogRollBackupSubprocedure extends Subprocedure { + private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedure.class); + + private final RegionServerServices rss; + private final LogRollBackupSubprocedurePool taskManager; + private FSHLog hlog; + + public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member, + ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout, + LogRollBackupSubprocedurePool taskManager) { + + super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener, + wakeFrequency, timeout); + LOG.info("Constructing a LogRollBackupSubprocedure."); + this.rss = rss; + this.taskManager = taskManager; + } + + /** + * Callable task. TODO. We don't need a thread pool to execute roll log. This can be simplified + * with no use of subprocedurepool. + */ + class RSRollLogTask implements Callable { + RSRollLogTask() { + } + + @Override + public Void call() throws Exception { + if (LOG.isDebugEnabled()) { + LOG.debug("++ DRPC started: " + rss.getServerName()); + } + hlog = (FSHLog) rss.getWAL(null); + long filenum = hlog.getFilenum(); + + LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum); + hlog.rollWriter(true); + LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum()); + // write the log number to hbase:backup. + BackupSystemTable table = BackupSystemTable.getTable(rss.getConfiguration()); + // sanity check, good for testing + HashMap serverTimestampMap = table.readRegionServerLastLogRollResult(); + String host = rss.getServerName().getHostname(); + String sts = serverTimestampMap.get(host); + if (sts != null && Long.parseLong(sts) > filenum) { + LOG.warn("Won't update server's last roll log result: current=" + sts + " new=" + filenum); + return null; + } + table.writeRegionServerLastLogRollResult(host, Long.toString(filenum)); + // TODO: potential leak of HBase connection + // BackupSystemTable.close(); + return null; + } + + } + + 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() throws ForeignException { + // do nothing, executing in inside barrier step. + } + + /** + * do a log roll. + * @return some bytes + */ + @Override + public byte[] insideBarrier() throws ForeignException { + rolllog(); + // FIXME + 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-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java new file mode 100644 index 00000000000..1ca638ce20f --- /dev/null +++ b/hbase-server/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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.DaemonThreadFactory; +import org.apache.hadoop.hbase.errorhandling.ForeignException; + +/** + * Handle running each of the individual tasks for completing a backup procedure + * on a regionserver. + */ +public class LogRollBackupSubprocedurePool implements Closeable, Abortable { + private static final Log LOG = LogFactory.getLog(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 DaemonThreadFactory("rs(" + name + + ")-backup-pool")); + 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; + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java new file mode 100644 index 00000000000..aca190c51de --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java @@ -0,0 +1,168 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +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.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; + +/** + * This manager class handles the work dealing with backup for a {@link HRegionServer}. + *

+ * 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 subprocedure, 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 + */ +public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager { + + private static final Log LOG = LogFactory.getLog(LogRollRegionServerProcedureManager.class); + + /** Conf key for number of request threads to start backup on regionservers */ + 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; + + /** + * Create a default backup procedure manager + */ + public LogRollRegionServerProcedureManager() { + } + + /** + * Start accepting backup procedure requests. + */ + @Override + public void start() { + this.memberRpcs.start(rss.getServerName().toString(), member); + 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 { + 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 ProcedureMemeber. + */ + public Subprocedure buildSubprocedure() { + + // 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); + + } + + /** + * 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(); + } + } + + @Override + public void initialize(RegionServerServices rss) throws IOException { + this.rss = rss; + BaseCoordinatedStateManager coordManager = + (BaseCoordinatedStateManager) CoordinatedStateManagerFactory.getCoordinatedStateManager(rss + .getConfiguration()); + coordManager.initialize(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-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java index ae36f08082e..3342743e028 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java @@ -17,7 +17,11 @@ */ package org.apache.hadoop.hbase.coordination; +import java.io.IOException; + import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.Server; @@ -51,8 +55,21 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan * Method to retrieve coordination for split log worker */ public abstract SplitLogWorkerCoordination getSplitLogWorkerCoordination(); + /** * Method to retrieve coordination for split log manager */ public abstract SplitLogManagerCoordination getSplitLogManagerCoordination(); + /** + * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs} + */ + public abstract ProcedureCoordinatorRpcs + getProcedureCoordinatorRpcs(String procType, String coordNode) throws IOException; + + /** + * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureMemberRpc} + */ + public abstract ProcedureMemberRpcs + getProcedureMemberRpcs(String procType) throws IOException; + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java index 3e89be77e0c..7cf4aabcc04 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java @@ -17,9 +17,15 @@ */ package org.apache.hadoop.hbase.coordination; +import java.io.IOException; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs; +import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; /** @@ -49,9 +55,21 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager { @Override public SplitLogWorkerCoordination getSplitLogWorkerCoordination() { return splitLogWorkerCoordination; - } + } + @Override public SplitLogManagerCoordination getSplitLogManagerCoordination() { return splitLogManagerCoordination; } + + @Override + public ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode) + throws IOException { + return new ZKProcedureCoordinatorRpcs(watcher, procType, coordNode); + } + + @Override + public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws IOException { + return new ZKProcedureMemberRpcs(watcher, procType); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index 9d9cee0e8a8..2ceeda5901d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -85,6 +85,9 @@ public class WALPlayer extends Configured implements Tool { private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; + public WALPlayer(){ + } + protected WALPlayer(final Configuration c) { super(c); } @@ -94,7 +97,7 @@ public class WALPlayer extends Configured implements Tool { * This one can be used together with {@link KeyValueSortReducer} */ static class WALKeyValueMapper - extends Mapper { + extends Mapper { private byte[] table; @Override @@ -106,7 +109,9 @@ public class WALPlayer extends Configured implements Tool { if (Bytes.equals(table, key.getTablename().getName())) { for (Cell cell : value.getCells()) { KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - if (WALEdit.isMetaEditFamily(kv)) continue; + if (WALEdit.isMetaEditFamily(kv)) { + continue; + } context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv); } } @@ -132,7 +137,7 @@ public class WALPlayer extends Configured implements Tool { * a running HBase instance. */ protected static class WALMapper - extends Mapper { + extends Mapper { private Map tables = new TreeMap(); @Override @@ -149,7 +154,9 @@ public class WALPlayer extends Configured implements Tool { Cell lastCell = null; for (Cell cell : value.getCells()) { // filtering WAL meta entries - if (WALEdit.isMetaEditFamily(cell)) continue; + if (WALEdit.isMetaEditFamily(cell)) { + continue; + } // Allow a subclass filter out this cell. if (filter(context, cell)) { @@ -160,8 +167,12 @@ public class WALPlayer extends Configured implements Tool { if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte() || !CellUtil.matchingRow(lastCell, cell)) { // row or type changed, write out aggregate KVs. - if (put != null) context.write(tableOut, put); - if (del != null) context.write(tableOut, del); + if (put != null) { + context.write(tableOut, put); + } + if (del != null) { + context.write(tableOut, del); + } if (CellUtil.isDelete(cell)) { del = new Delete(CellUtil.cloneRow(cell)); } else { @@ -177,8 +188,12 @@ public class WALPlayer extends Configured implements Tool { lastCell = cell; } // write residual KVs - if (put != null) context.write(tableOut, put); - if (del != null) context.write(tableOut, del); + if (put != null) { + context.write(tableOut, put); + } + if (del != null) { + context.write(tableOut, del); + } } } catch (InterruptedException e) { e.printStackTrace(); @@ -186,7 +201,8 @@ public class WALPlayer extends Configured implements Tool { } /** - * @param cell + * Filter cell + * @param cell cell * @return Return true if we are to emit this cell. */ protected boolean filter(Context context, final Cell cell) { @@ -197,9 +213,7 @@ public class WALPlayer extends Configured implements Tool { public void setup(Context context) throws IOException { String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY); String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY); - if (tablesToUse == null && tableMap == null) { - // Then user wants all tables. - } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) { + if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) { // this can only happen when WALMapper is used directly by a class other than WALPlayer throw new IOException("No tables or incorrect table mapping specified."); } @@ -215,7 +229,9 @@ public class WALPlayer extends Configured implements Tool { void setupTime(Configuration conf, String option) throws IOException { String val = conf.get(option); - if (null == val) return; + if (null == val) { + return; + } long ms; try { // first try to parse in user friendly form @@ -295,7 +311,8 @@ public class WALPlayer extends Configured implements Tool { return job; } - /* + /** + * Print usage * @param errorMsg Error message. Can be null. */ private void usage(final String errorMsg) { @@ -305,7 +322,8 @@ public class WALPlayer extends Configured implements Tool { System.err.println("Usage: " + NAME + " [options] []"); System.err.println("Read all WAL entries for ."); System.err.println("If no tables (\"\") are specific, all tables are imported."); - System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)"); + System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported"+ + " in that case.)"); System.err.println("Otherwise is a comma separated list of tables.\n"); System.err.println("The WAL entries can be mapped to new set of tables via ."); System.err.println(" is a command separated list of targettables."); @@ -318,10 +336,10 @@ public class WALPlayer extends Configured implements Tool { System.err.println(" -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]"); System.err.println(" -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]"); System.err.println(" -D " + JOB_NAME_CONF_KEY - + "=jobName - use the specified mapreduce job name for the wal player"); + + "=jobName - use the specified mapreduce job name for the wal player"); System.err.println("For performance also consider the following options:\n" - + " -Dmapreduce.map.speculative=false\n" - + " -Dmapreduce.reduce.speculative=false"); + + " -Dmapreduce.map.speculative=false\n" + + " -Dmapreduce.reduce.speculative=false"); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index bdb19f436ef..5cd38b54da6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; +import org.apache.hadoop.hbase.backup.BackupManager; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; @@ -384,6 +385,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.conf.setBoolean(HConstants.USE_META_REPLICAS, false); Replication.decorateMasterConfiguration(this.conf); + BackupManager.decorateMasterConfiguration(this.conf); // Hack! Maps DFSClient => Master for logs. HDFS made this // config param for task trackers, but we can piggyback off of it. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java index 95c3ffe918a..b6e11ea47d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java @@ -37,7 +37,7 @@ public abstract class RegionServerProcedureManager extends ProcedureManager { * @param rss Region Server service interface * @throws KeeperException */ - public abstract void initialize(RegionServerServices rss) throws KeeperException; + public abstract void initialize(RegionServerServices rss) throws IOException; /** * Start accepting procedure requests. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java index 0f4ea645869..adb3604b3b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager; -import org.apache.zookeeper.KeeperException; /** * Provides the globally barriered procedure framework and environment @@ -39,7 +38,7 @@ public class RegionServerProcedureManagerHost extends private static final Log LOG = LogFactory .getLog(RegionServerProcedureManagerHost.class); - public void initialize(RegionServerServices rss) throws KeeperException { + public void initialize(RegionServerServices rss) throws IOException { for (RegionServerProcedureManager proc : procedures) { LOG.debug("Procedure " + proc.getProcedureSignature() + " is initializing"); proc.initialize(rss); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java index 085d642c9ec..3865ba9d847 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java @@ -54,7 +54,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { * @throws KeeperException if an unexpected zk error occurs */ public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher, - String procedureClass, String coordName) throws KeeperException { + String procedureClass, String coordName) throws IOException { this.watcher = watcher; this.procedureType = procedureClass; this.coordName = coordName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java index 2e03a600229..fff75a7a5de 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java @@ -68,49 +68,54 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { * @throws KeeperException if we can't reach zookeeper */ public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType) - throws KeeperException { - this.zkController = new ZKProcedureUtil(watcher, procType) { - @Override - public void nodeCreated(String path) { - if (!isInProcedurePath(path)) { - return; + throws IOException { + try { + this.zkController = new ZKProcedureUtil(watcher, procType) { + @Override + public void nodeCreated(String path) { + if (!isInProcedurePath(path)) { + return; + } + + LOG.info("Received created event:" + path); + // if it is a simple start/end/abort then we just rewatch the node + if (isAcquiredNode(path)) { + waitForNewProcedures(); + return; + } else if (isAbortNode(path)) { + watchForAbortedProcedures(); + return; + } + String parent = ZKUtil.getParent(path); + // if its the end barrier, the procedure can be completed + if (isReachedNode(parent)) { + receivedReachedGlobalBarrier(path); + return; + } else if (isAbortNode(parent)) { + abort(path); + return; + } else if (isAcquiredNode(parent)) { + startNewSubprocedure(path); + } else { + LOG.debug("Ignoring created notification for node:" + path); + } } - LOG.info("Received created event:" + path); - // if it is a simple start/end/abort then we just rewatch the node - if (isAcquiredNode(path)) { - waitForNewProcedures(); - return; - } else if (isAbortNode(path)) { - watchForAbortedProcedures(); - return; + @Override + public void nodeChildrenChanged(String path) { + if (path.equals(this.acquiredZnode)) { + LOG.info("Received procedure start children changed event: " + path); + waitForNewProcedures(); + } else if (path.equals(this.abortZnode)) { + LOG.info("Received procedure abort children changed event: " + path); + watchForAbortedProcedures(); + } } - String parent = ZKUtil.getParent(path); - // if its the end barrier, the procedure can be completed - if (isReachedNode(parent)) { - receivedReachedGlobalBarrier(path); - return; - } else if (isAbortNode(parent)) { - abort(path); - return; - } else if (isAcquiredNode(parent)) { - startNewSubprocedure(path); - } else { - LOG.debug("Ignoring created notification for node:" + path); - } - } - - @Override - public void nodeChildrenChanged(String path) { - if (path.equals(this.acquiredZnode)) { - LOG.info("Received procedure start children changed event: " + path); - waitForNewProcedures(); - } else if (path.equals(this.abortZnode)) { - LOG.info("Received procedure abort children changed event: " + path); - watchForAbortedProcedures(); - } - } - }; + }; + } catch (KeeperException e) { + // TODO Auto-generated catch block + throw new IOException(e); + } } public ZKProcedureUtil getZkController() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java index 1aa959cad37..bd65cc7208b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java @@ -317,7 +317,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur * @throws KeeperException if the zookeeper cannot be reached */ @Override - public void initialize(RegionServerServices rss) throws KeeperException { + public void initialize(RegionServerServices rss) throws IOException { this.rss = rss; ZooKeeperWatcher zkw = rss.getZooKeeper(); this.memberRpcs = new ZKProcedureMemberRpcs(zkw, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 211fed5ebfc..1cd54fa4a41 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -807,8 +807,8 @@ public class HRegionServer extends HasThread implements rspmHost = new RegionServerProcedureManagerHost(); rspmHost.loadProcedures(conf); rspmHost.initialize(this); - } catch (KeeperException e) { - this.abort("Failed to reach zk cluster when creating procedure handler.", e); + } catch (IOException e) { + this.abort("Failed to reach coordination cluster when creating procedure handler.", e); } // register watcher for recovering regions this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java index 537329af8a6..e56dd2806f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java @@ -390,7 +390,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager { * @throws KeeperException if the zookeeper cluster cannot be reached */ @Override - public void initialize(RegionServerServices rss) throws KeeperException { + public void initialize(RegionServerServices rss) throws IOException { this.rss = rss; ZooKeeperWatcher zkw = rss.getZooKeeper(); this.memberRpcs = new ZKProcedureMemberRpcs(zkw, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 9ae72e64569..acde21e2ed7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -97,6 +97,8 @@ import com.lmax.disruptor.TimeoutException; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; + + /** * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. * Only one WAL is ever being written at a time. When a WAL hits a configured maximum size, @@ -359,7 +361,9 @@ public class FSHLog implements WAL { public int compare(Path o1, Path o2) { long t1 = getFileNumFromFileName(o1); long t2 = getFileNumFromFileName(o2); - if (t1 == t2) return 0; + if (t1 == t2) { + return 0; + } return (t1 > t2) ? 1 : -1; } }; @@ -402,7 +406,7 @@ public class FSHLog implements WAL { * @param root path for stored and archived wals * @param logDir dir where wals are stored * @param conf configuration to use - * @throws IOException + * @throws IOException exception */ public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf) throws IOException { @@ -410,7 +414,7 @@ public class FSHLog implements WAL { } /** - * Create an edit log at the given dir location. + * Create an edit log at the given directory location. * * You should never have to load an existing log. If there is a log at * startup, it should have already been processed and deleted by the time the @@ -425,13 +429,13 @@ public class FSHLog implements WAL { * be registered before we do anything else; e.g. the * Constructor {@link #rollWriter()}. * @param failIfWALExists If true IOException will be thrown if files related to this wal - * already exist. + * already exist. * @param prefix should always be hostname and port in distributed env and - * it will be URL encoded before being used. - * If prefix is null, "wal" will be used + * it will be URL encoded before being used. + * If prefix is null, "wal" will be used * @param suffix will be url encoded. null is treated as empty. non-empty must start with - * {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER} - * @throws IOException + * {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER} + * @throws IOException exception */ public FSHLog(final FileSystem fs, final Path rootDir, final String logDir, final String archiveDir, final Configuration conf, @@ -593,7 +597,9 @@ public class FSHLog implements WAL { @VisibleForTesting OutputStream getOutputStream() { FSDataOutputStream fsdos = this.hdfs_out; - if (fsdos == null) return null; + if (fsdos == null) { + return null; + } return fsdos.getWrappedStream(); } @@ -628,7 +634,7 @@ public class FSHLog implements WAL { /** * Tell listeners about pre log roll. - * @throws IOException + * @throws IOException exception */ private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath) throws IOException { @@ -641,7 +647,7 @@ public class FSHLog implements WAL { /** * Tell listeners about post log roll. - * @throws IOException + * @throws IOException exception */ private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath) throws IOException { @@ -654,8 +660,7 @@ public class FSHLog implements WAL { /** * Run a sync after opening to set up the pipeline. - * @param nextWriter - * @param startTimeNanos + * @param nextWriter next writer */ private void preemptiveSync(final ProtobufLogWriter nextWriter) { long startTimeNanos = System.nanoTime(); @@ -673,7 +678,9 @@ public class FSHLog implements WAL { rollWriterLock.lock(); try { // Return if nothing to flush. - if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null; + if (!force && (this.writer != null && this.numEntries.get() <= 0)) { + return null; + } byte [][] regionsToFlush = null; if (this.closed) { LOG.debug("WAL closed. Skipping rolling of writer"); @@ -728,7 +735,7 @@ public class FSHLog implements WAL { /** * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed. - * @throws IOException + * @throws IOException exception */ private void cleanOldLogs() throws IOException { List logsToArchive = null; @@ -738,9 +745,13 @@ public class FSHLog implements WAL { Path log = e.getKey(); Map sequenceNums = e.getValue(); if (this.sequenceIdAccounting.areAllLower(sequenceNums)) { - if (logsToArchive == null) logsToArchive = new ArrayList(); + if (logsToArchive == null) { + logsToArchive = new ArrayList(); + } logsToArchive.add(log); - if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log); + if (LOG.isTraceEnabled()) { + LOG.trace("WAL file ready for archiving " + log); + } } } if (logsToArchive != null) { @@ -770,7 +781,9 @@ public class FSHLog implements WAL { if (regions != null) { StringBuilder sb = new StringBuilder(); for (int i = 0; i < regions.length; i++) { - if (i > 0) sb.append(", "); + if (i > 0) { + sb.append(", "); + } sb.append(Bytes.toStringBinary(regions[i])); } LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs + @@ -836,7 +849,9 @@ public class FSHLog implements WAL { } } catch (FailedSyncBeforeLogCloseException e) { // If unflushed/unsynced entries on close, it is reason to abort. - if (isUnflushedEntries()) throw e; + if (isUnflushedEntries()) { + throw e; + } LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " + e.getMessage()); } @@ -897,7 +912,9 @@ public class FSHLog implements WAL { try { blockOnSync(syncFuture); } catch (IOException ioe) { - if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe); + if (LOG.isTraceEnabled()) { + LOG.trace("Stale sync exception", ioe); + } } } } @@ -968,7 +985,15 @@ public class FSHLog implements WAL { public Path getCurrentFileName() { return computeFilename(this.filenum.get()); } - + + /** + * To support old API compatibility + * @return current file number (timestamp) + */ + public long getFilenum() { + return filenum.get(); + } + @Override public String toString() { return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java new file mode 100644 index 00000000000..036000045f0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.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.snapshot; + +import org.apache.hadoop.hbase.backup.BackupHandler; + +/* this class will be extended in future jira to support progress report */ +public class SnapshotCopy extends ExportSnapshot { + private BackupHandler backupHandler; + private String table; + + public SnapshotCopy(BackupHandler backupHandler, String table) { + super(); + this.backupHandler = backupHandler; + this.table = table; + } + + public BackupHandler getBackupHandler() { + return this.backupHandler; + } + + public String getTable() { + return this.table; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java index 027e7a21306..dd4d3376c14 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java @@ -209,13 +209,18 @@ public class DefaultWALProvider implements WALProvider { @VisibleForTesting public static long extractFileNumFromWAL(final WAL wal) { final Path walName = ((FSHLog)wal).getCurrentFileName(); + return extractFileNumFromWAL(walName); + } + + @VisibleForTesting + public static long extractFileNumFromWAL(final Path walName) { if (walName == null) { throw new IllegalArgumentException("The WAL path couldn't be null"); } final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER); return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]); } - + /** * Pattern used to validate a WAL file name * see {@link #validateWALFilename(String)} for description. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java new file mode 100644 index 00000000000..bc0c84800ef --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -0,0 +1,194 @@ +/* + * + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +/** + * 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 Log LOG = LogFactory.getLog(TestBackupBase.class); + + protected static Configuration conf1; + protected static Configuration conf2; + + protected static HBaseTestingUtility TEST_UTIL; + protected static HBaseTestingUtility TEST_UTIL2; + + protected static TableName table1; + protected static TableName table2; + protected static TableName table3; + protected static TableName table4; + + protected static String table1_restore = "table1_restore"; + protected static String table2_restore = "table2_restore"; + protected static String table3_restore = "table3_restore"; + protected static String table4_restore = "table4_restore"; + + protected static final int NB_ROWS_IN_BATCH = 100; + protected static final byte[] qualName = Bytes.toBytes("q1"); + protected static final byte[] famName = Bytes.toBytes("f"); + + protected static String BACKUP_ROOT_DIR = "/backupUT"; + protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT"; + + protected static final String BACKUP_ZNODE = "/backup/hbase"; + protected static final String BACKUP_SUCCEED_NODE = "complete"; + protected static final String BACKUP_FAILED_NODE = "failed"; + + + /** + * @throws java.lang.Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + + TEST_UTIL = new HBaseTestingUtility(); + TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes", + LogRollRegionServerProcedureManager.class.getName()); + TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes", + LogRollMasterProcedureManager.class.getName()); + TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); + TEST_UTIL.startMiniZKCluster(); + MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster(); + + conf1 = TEST_UTIL.getConfiguration(); + conf2 = HBaseConfiguration.create(conf1); + conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2"); + TEST_UTIL2 = new HBaseTestingUtility(conf2); + TEST_UTIL2.setZkCluster(miniZK); + TEST_UTIL.startMiniCluster(); + TEST_UTIL2.startMiniCluster(); + conf1 = TEST_UTIL.getConfiguration(); + + TEST_UTIL.startMiniMapReduceCluster(); + BACKUP_ROOT_DIR = TEST_UTIL.getConfiguration().get("fs.defaultFS") + "/backupUT"; + LOG.info("ROOTDIR " + BACKUP_ROOT_DIR); + BACKUP_REMOTE_ROOT_DIR = TEST_UTIL2.getConfiguration().get("fs.defaultFS") + "/backupUT"; + LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR); + + BackupClient.setConf(conf1); + RestoreClient.setConf(conf1); + createTables(); + } + + /** + * @throws java.lang.Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin()); + SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); + //zkw1.close(); + TEST_UTIL2.shutdownMiniCluster(); + TEST_UTIL.shutdownMiniCluster(); + TEST_UTIL.shutdownMiniMapReduceCluster(); + } + + protected static void loadTable(HTable 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.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + table.put(p); + } + } + + protected static void createTables() throws Exception { + + long tid = System.currentTimeMillis(); + table1 = TableName.valueOf("test-" + tid); + HBaseAdmin ha = TEST_UTIL.getHBaseAdmin(); + HTableDescriptor desc = new HTableDescriptor(table1); + HColumnDescriptor fam = new HColumnDescriptor(famName); + desc.addFamily(fam); + ha.createTable(desc); + Connection conn = ConnectionFactory.createConnection(conf1); + HTable table = (HTable) conn.getTable(table1); + loadTable(table); + table.close(); + table2 = TableName.valueOf("test-" + tid + 1); + desc = new HTableDescriptor(table2); + desc.addFamily(fam); + ha.createTable(desc); + table = (HTable) conn.getTable(table2); + loadTable(table); + table.close(); + table3 = TableName.valueOf("test-" + tid + 2); + table = TEST_UTIL.createTable(table3, famName); + table.close(); + table4 = TableName.valueOf("test-" + tid + 3); + table = TEST_UTIL.createTable(table4, famName); + table.close(); + ha.close(); + conn.close(); + } + + protected boolean checkSucceeded(String backupId) throws IOException + { + BackupContext status = getBackupContext(backupId); + if(status == null) return false; + return status.getFlag() == BACKUPSTATUS.COMPLETE; + } + + protected boolean checkFailed(String backupId) throws IOException + { + BackupContext status = getBackupContext(backupId); + if(status == null) return false; + return status.getFlag() == BACKUPSTATUS.FAILED; + } + + private BackupContext getBackupContext(String backupId) throws IOException + { + Configuration conf = BackupClient.getConf(); + BackupSystemTable table = BackupSystemTable.getTable(conf); + BackupContext status = table.readBackupStatus(backupId); + return status; + } +} + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java new file mode 100644 index 00000000000..8be07bc6bae --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java @@ -0,0 +1,99 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestBackupBoundaryTests extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupBoundaryTests.class); + + /** + * Verify that full backup is created on a single empty table correctly. + * @throws Exception + */ + @Test + public void testFullBackupSingleEmpty() throws Exception { + + LOG.info("create full backup image on single table"); + + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table3.getNameAsString(), null); + LOG.info("Finished Backup"); + assertTrue(checkSucceeded(backupId)); + } + + /** + * Verify that full backup is created on multiple empty tables correctly. + * @throws Exception + */ + @Test + public void testFullBackupMultipleEmpty() throws Exception { + LOG.info("create full backup image on mulitple empty tables"); + String tableset = + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table4.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + + } + + /** + * Verify that full backup fails on a single table that does not exist. + * @throws Exception + */ + @Test(expected = RuntimeException.class) + public void testFullBackupSingleDNE() throws Exception { + + LOG.info("test full backup fails on a single table that does not exist"); + BackupClient.create("full", BACKUP_ROOT_DIR, "tabledne", null); + } + + /** + * Verify that full backup fails on multiple tables that do not exist. + * @throws Exception + */ + @Test(expected = RuntimeException.class) + public void testFullBackupMultipleDNE() throws Exception { + + LOG.info("test full backup fails on multiple tables that do not exist"); + BackupClient.create("full", BACKUP_ROOT_DIR, "table1dne,table2dne", null); + } + + /** + * Verify that full backup fails on tableset containing real and fake tables. + * @throws Exception + */ + @Test(expected = RuntimeException.class) + public void testFullBackupMixExistAndDNE() throws Exception { + LOG.info("create full backup fails on tableset containing real and fake table"); + String tableset = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + "tabledne"; + BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java new file mode 100644 index 00000000000..158479bcbde --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.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.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.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.HConstants; +import org.apache.hadoop.hbase.backup.master.BackupLogCleaner; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Iterables; + +@Category(LargeTests.class) +public class TestBackupLogCleaner extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(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"); + String tablesetFull = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table4.getNameAsString(); + + BackupSystemTable systemTable = BackupSystemTable.getTable(TEST_UTIL.getConfiguration()); + // Verify that we have no backup sessions yet + assertFalse(systemTable.hasBackupSessions()); + + List walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + List swalFiles = convert(walFiles); + BackupLogCleaner cleaner = new BackupLogCleaner(); + cleaner.setConf(TEST_UTIL.getConfiguration()); + + Iterable deletable = cleaner.getDeletableFiles(walFiles); + // We can delete all files because we do not have yet recorded backup sessions + assertTrue(Iterables.size(deletable) == walFiles.size()); + + systemTable.addWALFiles(swalFiles, "backup"); + String backupIdFull = BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null); + assertTrue(checkSucceeded(backupIdFull)); + // Check one more time + deletable = cleaner.getDeletableFiles(walFiles); + // We can delete wal files because they were saved into hbase:backup table + int size = Iterables.size(deletable); + assertTrue(size == walFiles.size()); + + List newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + LOG.debug("WAL list after full backup"); + convert(newWalFiles); + + // 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()); + // TODO : verify that result files are not walFiles collection + Connection conn = ConnectionFactory.createConnection(conf1); + // #2 - insert some data to table + HTable t1 = (HTable) 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(); + + HTable t2 = (HTable) 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 + String tablesetIncMultiple = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + + String backupIdIncMultiple = + BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncMultiple, null); + assertTrue(checkSucceeded(backupIdIncMultiple)); + deletable = cleaner.getDeletableFiles(newWalFiles); + + assertTrue(Iterables.size(deletable) == newWalFiles.size()); + + conn.close(); + + } + + private List convert(List walFiles) { + List result = new ArrayList(); + for (FileStatus fs : walFiles) { + LOG.debug("+++WAL: " + fs.getPath().toString()); + result.add(fs.getPath().toString()); + } + return result; + } + + private List getListOfWALFiles(Configuration c) throws IOException { + Path logRoot = new Path(FSUtils.getRootDir(c), HConstants.HREGION_LOGDIR_NAME); + FileSystem fs = FileSystem.get(c); + RemoteIterator it = fs.listFiles(logRoot, true); + List logFiles = new ArrayList(); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (lfs.isFile() && !DefaultWALProvider.isMetaFile(lfs.getPath())) { + logFiles.add(lfs); + LOG.info(lfs); + } + } + return logFiles; + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java new file mode 100644 index 00000000000..742170758d8 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java @@ -0,0 +1,341 @@ +/** + * + * 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.assertFalse; +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.Iterator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS; +import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test cases for hbase:backup API + * + */ +@Category(MediumTests.class) +public class TestBackupSystemTable { + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + protected static Configuration conf = UTIL.getConfiguration(); + protected static MiniHBaseCluster cluster; + + @BeforeClass + public static void setUp() throws Exception { + cluster = UTIL.startMiniCluster(); + + } + + @Test + public void testUpdateReadDeleteBackupStatus() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + BackupContext ctx = createBackupContext(); + table.updateBackupStatus(ctx); + BackupContext readCtx = table.readBackupStatus(ctx.getBackupId()); + assertTrue(compare(ctx, readCtx)); + + // try fake backup id + readCtx = table.readBackupStatus("fake"); + + assertNull(readCtx); + // delete backup context + table.deleteBackupStatus(ctx.getBackupId()); + readCtx = table.readBackupStatus(ctx.getBackupId()); + assertNull(readCtx); + cleanBackupTable(); + } + + @Test + public void testWriteReadBackupStartCode() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + String code = "100"; + table.writeBackupStartCode(code); + String readCode = table.readBackupStartCode(); + assertEquals(code, readCode); + cleanBackupTable(); + } + + private void cleanBackupTable() throws IOException { + Admin admin = UTIL.getHBaseAdmin(); + admin.disableTable(BackupSystemTable.getTableName()); + admin.truncateTable(BackupSystemTable.getTableName(), true); + if (admin.isTableDisabled(BackupSystemTable.getTableName())) { + admin.enableTable(BackupSystemTable.getTableName()); + } + } + + @Test + public void testBackupHistory() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + int n = 10; + List list = createBackupContextList(n); + + // Load data + for (BackupContext bc : list) { + // Make sure we set right status + bc.setFlag(BACKUPSTATUS.COMPLETE); + table.updateBackupStatus(bc); + } + + // Reverse list for comparison + Collections.reverse(list); + ArrayList history = table.getBackupHistory(); + assertTrue(history.size() == n); + + for (int i = 0; i < n; i++) { + BackupContext ctx = list.get(i); + BackupCompleteData data = history.get(i); + assertTrue(compare(ctx, data)); + } + + cleanBackupTable(); + + } + + @Test + public void testRegionServerLastLogRollResults() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + + String[] servers = new String[] { "server1", "server2", "server3" }; + String[] timestamps = new String[] { "100", "102", "107" }; + + for (int i = 0; i < servers.length; i++) { + table.writeRegionServerLastLogRollResult(servers[i], timestamps[i]); + } + + HashMap result = table.readRegionServerLastLogRollResult(); + 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]); + String ts1 = timestamps[i]; + String ts2 = result.get(keysAsArray[i]); + assertEquals(ts1, ts2); + } + + cleanBackupTable(); + + } + + @Test + public void testIncrementalBackupTableSet() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + + TreeSet tables1 = new TreeSet(); + + tables1.add("t1"); + tables1.add("t2"); + tables1.add("t3"); + + TreeSet tables2 = new TreeSet(); + + tables2.add("t3"); + tables2.add("t4"); + tables2.add("t5"); + + table.addIncrementalBackupTableSet(tables1); + TreeSet res1 = (TreeSet) table.getIncrementalBackupTableSet(); + assertTrue(tables1.size() == res1.size()); + Iterator desc1 = tables1.descendingIterator(); + Iterator desc2 = res1.descendingIterator(); + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + + table.addIncrementalBackupTableSet(tables2); + TreeSet res2 = (TreeSet) table.getIncrementalBackupTableSet(); + 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 { + BackupSystemTable table = BackupSystemTable.getTable(conf); + + TreeSet tables = new TreeSet(); + + tables.add("t1"); + tables.add("t2"); + tables.add("t3"); + + HashMap rsTimestampMap = new HashMap(); + + rsTimestampMap.put("rs1", "100"); + rsTimestampMap.put("rs2", "101"); + rsTimestampMap.put("rs3", "103"); + + table.writeRegionServerLogTimestamp(tables, rsTimestampMap); + + HashMap> result = table.readLogTimestampMap(); + + assertTrue(tables.size() == result.size()); + + for (String t : tables) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1"), "100"); + assertEquals(rstm.get("rs2"), "101"); + assertEquals(rstm.get("rs3"), "103"); + } + + Set tables1 = new TreeSet(); + + tables1.add("t3"); + tables1.add("t4"); + tables1.add("t5"); + + HashMap rsTimestampMap1 = new HashMap(); + + rsTimestampMap1.put("rs1", "200"); + rsTimestampMap1.put("rs2", "201"); + rsTimestampMap1.put("rs3", "203"); + + table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1); + + result = table.readLogTimestampMap(); + + assertTrue(5 == result.size()); + + for (String t : tables) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + if (t.equals("t3") == false) { + assertEquals(rstm.get("rs1"), "100"); + assertEquals(rstm.get("rs2"), "101"); + assertEquals(rstm.get("rs3"), "103"); + } else { + assertEquals(rstm.get("rs1"), "200"); + assertEquals(rstm.get("rs2"), "201"); + assertEquals(rstm.get("rs3"), "203"); + } + } + + for (String t : tables1) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1"), "200"); + assertEquals(rstm.get("rs2"), "201"); + assertEquals(rstm.get("rs3"), "203"); + } + + cleanBackupTable(); + + } + + @Test + public void testAddWALFiles() throws IOException { + BackupSystemTable table = BackupSystemTable.getTable(conf); + FileSystem fs = FileSystem.get(conf); + List files = + Arrays.asList("hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.1", + "hdfs://server/WALs/srv2,102,16666/srv2,102,16666.default.2", + "hdfs://server/WALs/srv3,103,17777/srv3,103,17777.default.3"); + String newFile = "hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.5"; + + table.addWALFiles(files, "backup"); + + assertTrue(table.checkWALFile(files.get(0))); + assertTrue(table.checkWALFile(files.get(1))); + assertTrue(table.checkWALFile(files.get(2))); + assertFalse(table.checkWALFile(newFile)); + + cleanBackupTable(); + } + + private boolean compare(BackupContext ctx, BackupCompleteData data) { + + return ctx.getBackupId().equals(data.getBackupToken()) + && ctx.getTargetRootDir().equals(data.getBackupRootPath()) + && ctx.getType().equals(data.getType()) + && ctx.getStartTs() == Long.parseLong(data.getStartTime()) + && ctx.getEndTs() == Long.parseLong(data.getEndTime()); + + } + + private boolean compare(BackupContext one, BackupContext two) { + return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType()) + && one.getTargetRootDir().equals(two.getTargetRootDir()) + && one.getStartTs() == two.getStartTs() && one.getEndTs() == two.getEndTs(); + } + + private BackupContext createBackupContext() { + + BackupContext ctxt = + new BackupContext("backup_" + System.nanoTime(), "full", new String[] { "t1", "t2", "t3" }, + "/hbase/backup", null); + ctxt.setStartTs(System.currentTimeMillis()); + ctxt.setEndTs(System.currentTimeMillis() + 1); + return ctxt; + } + + private List createBackupContextList(int size) { + List list = new ArrayList(); + for (int i = 0; i < size; i++) { + list.add(createBackupContext()); + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + return list; + } + + @AfterClass + public static void tearDown() throws IOException { + if (cluster != null) cluster.shutdown(); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java new file mode 100644 index 00000000000..19caf371b67 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java @@ -0,0 +1,83 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestFullBackup extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestFullBackup.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testFullBackupSingle() throws Exception { + + LOG.info("test full backup on a single table with data"); + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + } + + /** + * Verify that full backup is created on multiple tables correctly. + * @throws Exception + */ + @Test + public void testFullBackupMultiple() throws Exception { + LOG.info("create full backup image on multiple tables with data"); + String tableset = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table2.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + + } + + /** + * Verify that full backup is created on all tables correctly. + * @throws Exception + */ + @Test + public void testFullBackupAll() throws Exception { + LOG.info("create full backup image on all tables"); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, null, null); + assertTrue(checkSucceeded(backupId)); + + } + + /** + * Verify that full backup is created on a table correctly using a snapshot. + * @throws Exception + */ + //@Test + //public void testFullBackupUsingSnapshot() throws Exception { + // HBaseAdmin hba = new HBaseAdmin(conf1); + //String snapshot = "snapshot"; + //hba.snapshot(snapshot, table1); + //LOG.info("create full backup image on a table using snapshot"); + //String backupId = + // BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), + // snapshot); + // } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java new file mode 100644 index 00000000000..1262c54727c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java @@ -0,0 +1,166 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable + * law or agreed to in writing, software distributed under the License is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License + * for the specific language governing permissions and limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestFullRestore extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestFullRestore.class); + + /** + * Verify that a single table is restored to a new table + * @throws Exception + */ + @Test + public void testFullRestoreSingle() throws Exception { + + LOG.info("test full restore on a single table empty table"); + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + String[] tableset = new String[] { table1.getNameAsString() }; + String[] tablemap = new String[] { table1_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap, + false); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(TableName.valueOf(table1_restore))); + TEST_UTIL.deleteTable(TableName.valueOf(table1_restore)); + hba.close(); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception + */ + @Test + public void testFullRestoreMultiple() throws Exception { + LOG.info("create full backup image on multiple tables"); + String tableset = + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + + String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() }; + String[] tablemap = new String[] { table2_restore, table3_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, + restore_tableset, tablemap, false); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(TableName.valueOf(table2_restore))); + assertTrue(hba.tableExists(TableName.valueOf(table3_restore))); + TEST_UTIL.deleteTable(TableName.valueOf(table2_restore)); + TEST_UTIL.deleteTable(TableName.valueOf(table3_restore)); + hba.close(); + } + + /** + * Verify that a single table is restored using overwrite + * @throws Exception + */ + @Test + public void testFullRestoreSingleOverwrite() throws Exception { + + LOG.info("test full restore on a single table empty table"); + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + String[] tableset = new String[] { table1.getNameAsString() }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, null, + true); + } + + /** + * Verify that multiple tables are restored to new tables using overwrite. + * @throws Exception + */ + @Test + public void testFullRestoreMultipleOverwrite() throws Exception { + LOG.info("create full backup image on multiple tables"); + String tableset = + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + + String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, + false, restore_tableset, null, true); + } + + /** + * Verify that restore fails on a single table that does not exist. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullRestoreSingleDNE() throws Exception { + + LOG.info("test restore fails on a single table that does not exist"); + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + String[] tableset = new String[] { "faketable" }; + String[] tablemap = new String[] { table1_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap, + false); + } + + /** + * Verify that restore fails on multiple tables that do not exist. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullRestoreMultipleDNE() throws Exception { + + LOG.info("test restore fails on multiple tables that do not exist"); + String tableset = + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + + String[] restore_tableset = new String[] { "faketable1", "faketable2" }; + String[] tablemap = new String[] { table2_restore, table3_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, + false, restore_tableset, tablemap, false); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java new file mode 100644 index 00000000000..5437b8427f1 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.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; + +import static org.junit.Assert.assertTrue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.fs.Path; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestIncrementalBackup extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class); + //implement all testcases in 1 test since incremental backup/restore has dependencies + @Test + public void TestIncBackupRestore() throws Exception { + HBackupFileSystem hbfs; + + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + String tablesetFull = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table4.getNameAsString(); + + String backupIdFull = + BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null); + assertTrue(checkSucceeded(backupIdFull)); + + Connection conn = ConnectionFactory.createConnection(conf1); + // #2 - insert some data to table + HTable t1 = (HTable) 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.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2)); + t1.close(); + + HTable t2 = (HTable) 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.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5)); + t2.close(); + + // #3 - incremental backup for multiple tables + String tablesetIncMultiple = + table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + + String backupIdIncMultiple = BackupClient.create("incremental", BACKUP_ROOT_DIR, + tablesetIncMultiple, null); + assertTrue(checkSucceeded(backupIdIncMultiple)); + + + // #4 - restore full backup for all tables, without overwrite + String[] tablesRestoreFull = + new String[] { table1.getNameAsString(), table2.getNameAsString(), + table3.getNameAsString(), table4.getNameAsString() }; + + String[] tablesMapFull = + new String[] { table1_restore, table2_restore, table3_restore, table4_restore }; + + hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdFull); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdFull, false, false, + tablesRestoreFull, + tablesMapFull, false); + + // #5.1 - check tables for full restore + HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin(); + assertTrue(hAdmin.tableExists(TableName.valueOf(table1_restore))); + assertTrue(hAdmin.tableExists(TableName.valueOf(table2_restore))); + assertTrue(hAdmin.tableExists(TableName.valueOf(table3_restore))); + assertTrue(hAdmin.tableExists(TableName.valueOf(table4_restore))); + + hAdmin.close(); + + // #5.2 - checking row count of tables for full restore + HTable hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH)); + hTable.close(); + + hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH)); + hTable.close(); + + hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0)); + hTable.close(); + + hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0)); + hTable.close(); + + // #6 - restore incremental backup for multiple tables, with overwrite + String[] tablesRestoreIncMultiple = + new String[] + { table1.getNameAsString(), table2.getNameAsString(), table3.getNameAsString() }; + String[] tablesMapIncMultiple = + new String[] { table1_restore, table2_restore, table3_restore }; + hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncMultiple); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncMultiple, false, false, + tablesRestoreIncMultiple, tablesMapIncMultiple, true); + + hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2)); + hTable.close(); + + hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5)); + hTable.close(); + + hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0)); + hTable.close(); + + // #7 - incremental backup for single, empty table + + String tablesetIncEmpty = table4.getNameAsString(); + String backupIdIncEmpty = + BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncEmpty, null); + assertTrue(checkSucceeded(backupIdIncEmpty)); + + + // #8 - restore incremental backup for single empty table, with overwrite + String[] tablesRestoreIncEmpty = new String[] { table4.getNameAsString() }; + String[] tablesMapIncEmpty = new String[] { table4_restore }; + hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncEmpty); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncEmpty, false, false, + tablesRestoreIncEmpty, + tablesMapIncEmpty, true); + + hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore)); + Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0)); + hTable.close(); + conn.close(); + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java new file mode 100644 index 00000000000..03822a2feb7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java @@ -0,0 +1,44 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestRemoteBackup extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class); + + /** + * Verify that a remote full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testFullBackupRemote() throws Exception { + + LOG.info("test remote full backup on a single table"); + + // String rootdir = TEST_UTIL2.getDefaultRootDirPath() + BACKUP_ROOT_DIR; + // LOG.info("ROOTDIR " + rootdir); + String backupId = + BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java new file mode 100644 index 00000000000..e1315c5b5f6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java @@ -0,0 +1,54 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestRemoteRestore extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestRemoteRestore.class); + + /** + * Verify that a remote restore on a single table is successful. + * @throws Exception + */ + @Test + public void testFullRestoreRemote() throws Exception { + + LOG.info("test remote full backup on a single table"); + String backupId = + BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + String[] tableset = new String[] { table1.getNameAsString() }; + String[] tablemap = new String[] { table1_restore }; + Path path = new Path(BACKUP_REMOTE_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_REMOTE_ROOT_DIR, backupId, false, false, tableset, + tablemap, false); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(TableName.valueOf(table1_restore))); + TEST_UTIL.deleteTable(TableName.valueOf(table1_restore)); + hba.close(); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java new file mode 100644 index 00000000000..ad23f9ba915 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java @@ -0,0 +1,85 @@ +/** + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestRestoreBoundaryTests extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestRestoreBoundaryTests.class); + + /** + * Verify that a single empty table is restored to a new table + * @throws Exception + */ + @Test + public void testFullRestoreSingleEmpty() throws Exception { + + LOG.info("test full restore on a single table empty table"); + String backupId = + BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + String[] tableset = new String[] { table1.getNameAsString() }; + String[] tablemap = new String[] { table1_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap, + false); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(TableName.valueOf(table1_restore))); + TEST_UTIL.deleteTable(TableName.valueOf(table1_restore)); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception + */ + @Test + public void testFullRestoreMultipleEmpty() throws Exception { + LOG.info("create full backup image on multiple tables"); + String tableset = + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND + + table3.getNameAsString(); + String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null); + assertTrue(checkSucceeded(backupId)); + String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() }; + String[] tablemap = new String[] { table2_restore, table3_restore }; + Path path = new Path(BACKUP_ROOT_DIR); + HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId); + RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, restore_tableset, + tablemap, + false); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(TableName.valueOf(table2_restore))); + assertTrue(hba.tableExists(TableName.valueOf(table3_restore))); + TEST_UTIL.deleteTable(TableName.valueOf(table2_restore)); + TEST_UTIL.deleteTable(TableName.valueOf(table3_restore)); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java index 7620bbbecf8..cd2efad4708 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java @@ -49,7 +49,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager { private ProcedureMember member; @Override - public void initialize(RegionServerServices rss) throws KeeperException { + public void initialize(RegionServerServices rss) throws IOException { this.rss = rss; ZooKeeperWatcher zkw = rss.getZooKeeper(); this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());