diff --git a/bin/hbase b/bin/hbase index 1653c5a85a9..f1114af0591 100755 --- a/bin/hbase +++ b/bin/hbase @@ -103,6 +103,8 @@ if [ $# = 0 ]; then echo " ltt Run LoadTestTool" echo " canary Run the Canary tool" echo " version Print the version" + echo " backup Backup tables for recovery" + echo " restore Restore tables from existing backup image" echo " CLASSNAME Run the class named CLASSNAME" exit 1 fi @@ -315,6 +317,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.BackupDriver' +elif [ "$COMMAND" = "restore" ] ; then + CLASS='org.apache.hadoop.hbase.backup.RestoreDriver' 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-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 69181847e64..13680389604 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -4098,11 +4098,11 @@ public class HBaseAdmin implements Admin { if (result == 0) { Iterator remoteHCDIter = peerHtd.getFamilies().iterator(); Iterator localHCDIter = localHtd.getFamilies().iterator(); - + while (remoteHCDIter.hasNext() && localHCDIter.hasNext()) { HColumnDescriptor remoteHCD = remoteHCDIter.next(); HColumnDescriptor localHCD = localHCDIter.next(); - + String remoteHCDName = remoteHCD.getNameAsString(); String localHCDName = localHCD.getNameAsString(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java index 3f65e6e1777..c59b020587e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java @@ -107,6 +107,7 @@ public class RpcRetryingCallerImpl implements RpcRetryingCaller { } catch (PreemptiveFastFailException e) { throw e; } catch (Throwable t) { + Throwable e = t.getCause(); ExceptionUtil.rethrowIfInterrupt(t); // translateException throws exception when should not retry: i.e. when request is bad. @@ -185,7 +186,7 @@ public class RpcRetryingCallerImpl implements RpcRetryingCaller { } } } - + /** * Get the good or the remote exception if any, throws the DoNotRetryIOException. * @param t the throwable to analyze diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java new file mode 100644 index 00000000000..79f46369c6a --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java @@ -0,0 +1,25 @@ +/** + * 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; + +@InterfaceAudience.Private +public enum BackupType { + FULL, INCREMENTAL +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java index a51a80fba9d..29d10ae20ce 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java @@ -22,7 +22,9 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; +import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.MissingOptionException; @@ -32,9 +34,9 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -47,12 +49,15 @@ public abstract class AbstractHBaseTool implements Tool, Configurable { protected static final int EXIT_SUCCESS = 0; protected static final int EXIT_FAILURE = 1; + public static final String SHORT_HELP_OPTION = "h"; + public static final String LONG_HELP_OPTION = "help"; + private static final Option HELP_OPTION = new Option("h", "help", false, "Prints help for this tool."); private static final Log LOG = LogFactory.getLog(AbstractHBaseTool.class); - private final Options options = new Options(); + protected final Options options = new Options(); protected Configuration conf = null; @@ -108,7 +113,7 @@ public abstract class AbstractHBaseTool implements Tool, Configurable { } @Override - public final int run(String[] args) throws IOException { + public int run(String[] args) throws IOException { cmdLineArgs = args; if (conf == null) { LOG.error("Tool configuration is not initialized"); @@ -161,6 +166,13 @@ public abstract class AbstractHBaseTool implements Tool, Configurable { return cl.getOptions().length != 0; } + protected CommandLine parseArgs(String[] args) throws ParseException { + options.addOption(SHORT_HELP_OPTION, LONG_HELP_OPTION, false, "Show usage"); + addOptions(); + CommandLineParser parser = new BasicParser(); + return parser.parse(options, args); + } + protected void printUsage() { printUsage("hbase " + getClass().getName() + " ", "Options:", ""); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java new file mode 100644 index 00000000000..85964896524 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java @@ -0,0 +1,311 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; +import java.util.Set; + +import org.apache.commons.cli.CommandLine; +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.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.base.Objects; +import com.google.common.collect.Lists; + +/** + * An integration test to detect regressions in HBASE-7912. Create + * a table with many regions, load data, perform series backup/load operations, + * then restore and verify data + * @see HBASE-7912 + * @see HBASE-14123 + */ +@Category(IntegrationTests.class) +public class IntegrationTestBackupRestore extends IntegrationTestBase { + private static final String CLASS_NAME = IntegrationTestBackupRestore.class.getSimpleName(); + protected static final Log LOG = LogFactory.getLog(IntegrationTestBackupRestore.class); + protected static final TableName TABLE_NAME1 = TableName.valueOf(CLASS_NAME + ".table1"); + protected static final TableName TABLE_NAME2 = TableName.valueOf(CLASS_NAME + ".table2"); + protected static final String COLUMN_NAME = "f"; + protected static final String REGION_COUNT_KEY = "regions_per_rs"; + protected static final String REGIONSERVER_COUNT_KEY = "region_servers"; + protected static final int DEFAULT_REGION_COUNT = 10; + protected static final int DEFAULT_REGIONSERVER_COUNT = 2; + protected static int regionsCountPerServer; + protected static int regionServerCount; + protected static final String NB_ROWS_IN_BATCH_KEY = "rows_in_batch"; + protected static final int DEFAULT_NB_ROWS_IN_BATCH = 20000; + private static int rowsInBatch; + private static String BACKUP_ROOT_DIR = "backupIT"; + + @Override + @Before + public void setUp() throws Exception { + util = new IntegrationTestingUtility(); + Configuration conf = util.getConfiguration(); + regionsCountPerServer = conf.getInt(REGION_COUNT_KEY, DEFAULT_REGION_COUNT); + regionServerCount = + conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT); + rowsInBatch = conf.getInt(NB_ROWS_IN_BATCH_KEY, DEFAULT_NB_ROWS_IN_BATCH); + enableBackup(conf); + LOG.info(String.format("Initializing cluster with %d region servers.", regionServerCount)); + util.initializeCluster(regionServerCount); + LOG.info("Cluster initialized"); + util.deleteTableIfAny(TABLE_NAME1); + util.deleteTableIfAny(TABLE_NAME2); + LOG.info("Cluster ready"); + } + + @After + public void tearDown() throws IOException { + LOG.info("Cleaning up after test."); + if(util.isDistributedCluster()) { + util.deleteTableIfAny(TABLE_NAME1); + LOG.info("Cleaning up after test. TABLE1 done"); + util.deleteTableIfAny(TABLE_NAME2); + LOG.info("Cleaning up after test. TABLE2 done"); + cleanUpBackupDir(); + } + LOG.info("Restoring cluster."); + util.restoreCluster(); + LOG.info("Cluster restored."); + } + + private void enableBackup(Configuration conf) { + // Enable backup + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf); + BackupManager.decorateRegionServerConfiguration(conf); + } + + private void cleanUpBackupDir() throws IOException { + FileSystem fs = FileSystem.get(util.getConfiguration()); + fs.delete(new Path(BACKUP_ROOT_DIR), true); + } + + @Test + public void testBackupRestore() throws Exception { + BACKUP_ROOT_DIR = util.getDataTestDirOnTestFS() + Path.SEPARATOR + BACKUP_ROOT_DIR; + createTable(TABLE_NAME1); + createTable(TABLE_NAME2); + runTest(); + } + + + private void createTable(TableName tableName) throws Exception { + long startTime, endTime; + HTableDescriptor desc = new HTableDescriptor(tableName); + HColumnDescriptor[] columns = + new HColumnDescriptor[]{new HColumnDescriptor(COLUMN_NAME)}; + LOG.info(String.format("Creating table %s with %d splits.", tableName, + regionsCountPerServer)); + startTime = System.currentTimeMillis(); + HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(), desc, columns, + regionsCountPerServer); + util.waitTableAvailable(tableName); + endTime = System.currentTimeMillis(); + LOG.info(String.format("Pre-split table created successfully in %dms.", + (endTime - startTime))); + } + + private void loadData(TableName table, int numRows) throws IOException { + Connection conn = util.getConnection(); + // #0- insert some data to a table + Table t1 = conn.getTable(table); + util.loadRandomRows(t1, new byte[]{'f'}, 100, numRows); + // flush table + conn.getAdmin().flush(TableName.valueOf(table.getName())); + } + + private void runTest() throws IOException { + + try (Connection conn = util.getConnection(); + Admin admin = conn.getAdmin(); + BackupAdmin client = new BackupAdminImpl(conn);) { + + // #0- insert some data to table TABLE_NAME1, TABLE_NAME2 + loadData(TABLE_NAME1, rowsInBatch); + loadData(TABLE_NAME2, rowsInBatch); + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + List tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2); + + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = + builder.withBackupType(BackupType.FULL).withTableList(tables) + .withTargetRootDir(BACKUP_ROOT_DIR).build(); + String backupIdFull = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull)); + // #2 - insert some data to table + loadData(TABLE_NAME1, rowsInBatch); + loadData(TABLE_NAME2, rowsInBatch); + HTable t1 = (HTable) conn.getTable(TABLE_NAME1); + Assert.assertEquals(util.countRows(t1), rowsInBatch * 2); + t1.close(); + HTable t2 = (HTable) conn.getTable(TABLE_NAME2); + Assert.assertEquals(util.countRows(t2), rowsInBatch * 2); + t2.close(); + // #3 - incremental backup for tables + tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2); + builder = new BackupRequest.Builder(); + request = + builder.withBackupType(BackupType.INCREMENTAL).withTableList(tables) + .withTargetRootDir(BACKUP_ROOT_DIR).build(); + String backupIdIncMultiple = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple)); + // #4 - restore full backup for all tables, without overwrite + TableName[] tablesRestoreFull = new TableName[] { TABLE_NAME1, TABLE_NAME2 }; + client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, tablesRestoreFull, + null, true)); + // #5.1 - check tables for full restore + assertTrue(admin.tableExists(TABLE_NAME1)); + assertTrue(admin.tableExists(TABLE_NAME2)); + // #5.2 - checking row count of tables for full restore + HTable hTable = (HTable) conn.getTable(TABLE_NAME1); + Assert.assertEquals(util.countRows(hTable), rowsInBatch); + hTable.close(); + hTable = (HTable) conn.getTable(TABLE_NAME2); + Assert.assertEquals(util.countRows(hTable), rowsInBatch); + hTable.close(); + // #6 - restore incremental backup for multiple tables, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { TABLE_NAME1, TABLE_NAME2 }; + client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false, + tablesRestoreIncMultiple, null, true)); + hTable = (HTable) conn.getTable(TABLE_NAME1); + Assert.assertEquals(util.countRows(hTable), rowsInBatch * 2); + hTable.close(); + hTable = (HTable) conn.getTable(TABLE_NAME2); + Assert.assertEquals(util.countRows(hTable), rowsInBatch * 2); + hTable.close(); + } + } + + protected boolean checkSucceeded(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + if (status == null) return false; + return status.getState() == BackupState.COMPLETE; + } + + private BackupInfo getBackupInfo(String backupId) throws IOException { + try (BackupSystemTable table = new BackupSystemTable(util.getConnection())) { + return table.readBackupInfo(backupId); + } + } + + /** + * Get restore request. + */ + public RestoreRequest createRestoreRequest(String backupRootDir, String backupId, boolean check, + TableName[] fromTables, TableName[] toTables, boolean isOverwrite) { + RestoreRequest.Builder builder = new RestoreRequest.Builder(); + return builder.withBackupRootDir(backupRootDir) + .withBackupId(backupId) + .withCheck(check) + .withFromTables(fromTables) + .withToTables(toTables) + .withOvewrite(isOverwrite).build(); + } + + @Override + public void setUpCluster() throws Exception { + util = getTestingUtil(getConf()); + enableBackup(getConf()); + LOG.debug("Initializing/checking cluster has " + regionServerCount + " servers"); + util.initializeCluster(regionServerCount); + LOG.debug("Done initializing/checking cluster"); + } + + @Override + public int runTestFromCommandLine() throws Exception { + testBackupRestore(); + return 0; + } + + @Override + public TableName getTablename() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected Set getColumnFamilies() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected void addOptions() { + addOptWithArg(REGIONSERVER_COUNT_KEY, "Total number of region servers. Default: '" + + DEFAULT_REGIONSERVER_COUNT + "'"); + addOptWithArg(REGION_COUNT_KEY, "Total number of regions. Default: " + DEFAULT_REGION_COUNT); + addOptWithArg(NB_ROWS_IN_BATCH_KEY, "Total number of data rows to be loaded (per table/batch." + + " Total number of batches=2). Default: " + DEFAULT_NB_ROWS_IN_BATCH); + + } + + @Override + protected void processOptions(CommandLine cmd) { + super.processOptions(cmd); + regionsCountPerServer = + Integer.parseInt(cmd.getOptionValue(REGION_COUNT_KEY, + Integer.toString(DEFAULT_REGION_COUNT))); + regionServerCount = + Integer.parseInt(cmd.getOptionValue(REGIONSERVER_COUNT_KEY, + Integer.toString(DEFAULT_REGIONSERVER_COUNT))); + rowsInBatch = + Integer.parseInt(cmd.getOptionValue(NB_ROWS_IN_BATCH_KEY, + Integer.toString(DEFAULT_NB_ROWS_IN_BATCH))); + LOG.info(Objects.toStringHelper("Parsed Options").add(REGION_COUNT_KEY, regionsCountPerServer) + .add(REGIONSERVER_COUNT_KEY, regionServerCount).add(NB_ROWS_IN_BATCH_KEY, rowsInBatch) + .toString()); + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + IntegrationTestingUtility.setUseDistributedCluster(conf); + int status = ToolRunner.run(conf, new IntegrationTestBackupRestore(), args); + System.exit(status); + } +} diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java new file mode 100644 index 00000000000..4cad101507a --- /dev/null +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java @@ -0,0 +1,7013 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: Backup.proto + +package org.apache.hadoop.hbase.shaded.protobuf.generated; + +public final class BackupProtos { + private BackupProtos() {} + public static void registerAllExtensions( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry); + } + /** + * Protobuf enum {@code hbase.pb.BackupType} + */ + public enum BackupType + implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum { + /** + * FULL = 0; + */ + FULL(0), + /** + * INCREMENTAL = 1; + */ + INCREMENTAL(1), + ; + + /** + * FULL = 0; + */ + public static final int FULL_VALUE = 0; + /** + * INCREMENTAL = 1; + */ + public static final int INCREMENTAL_VALUE = 1; + + + public final int getNumber() { + return value; + } + + /** + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static BackupType valueOf(int value) { + return forNumber(value); + } + + public static BackupType forNumber(int value) { + switch (value) { + case 0: return FULL; + case 1: return INCREMENTAL; + default: return null; + } + } + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap< + BackupType> internalValueMap = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() { + public BackupType findValueByNumber(int number) { + return BackupType.forNumber(number); + } + }; + + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(ordinal()); + } + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0); + } + + private static final BackupType[] VALUES = values(); + + public static BackupType valueOf( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private BackupType(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType) + } + + public interface ServerTimestampOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.ServerTimestamp) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + boolean hasServerName(); + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName(); + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder(); + + /** + * optional uint64 timestamp = 2; + */ + boolean hasTimestamp(); + /** + * optional uint64 timestamp = 2; + */ + long getTimestamp(); + } + /** + *
+   **
+   * ServerTimestamp keeps last WAL roll time per Region Server
+   * 
+ * + * Protobuf type {@code hbase.pb.ServerTimestamp} + */ + public static final class ServerTimestamp extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.ServerTimestamp) + ServerTimestampOrBuilder { + // Use ServerTimestamp.newBuilder() to construct. + private ServerTimestamp(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private ServerTimestamp() { + timestamp_ = 0L; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ServerTimestamp( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = serverName_.toBuilder(); + } + serverName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(serverName_); + serverName_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + timestamp_ = input.readUInt64(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class); + } + + private int bitField0_; + public static final int SERVER_NAME_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_; + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public boolean hasServerName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName() { + return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_; + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() { + return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_; + } + + public static final int TIMESTAMP_FIELD_NUMBER = 2; + private long timestamp_; + /** + * optional uint64 timestamp = 2; + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 timestamp = 2; + */ + public long getTimestamp() { + return timestamp_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasServerName()) { + if (!getServerName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getServerName()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, timestamp_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getServerName()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, timestamp_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) obj; + + boolean result = true; + result = result && (hasServerName() == other.hasServerName()); + if (hasServerName()) { + result = result && getServerName() + .equals(other.getServerName()); + } + result = result && (hasTimestamp() == other.hasTimestamp()); + if (hasTimestamp()) { + result = result && (getTimestamp() + == other.getTimestamp()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasServerName()) { + hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER; + hash = (53 * hash) + getServerName().hashCode(); + } + if (hasTimestamp()) { + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getTimestamp()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     * ServerTimestamp keeps last WAL roll time per Region Server
+     * 
+ * + * Protobuf type {@code hbase.pb.ServerTimestamp} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.ServerTimestamp) + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServerNameFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serverNameBuilder_ == null) { + serverName_ = null; + } else { + serverNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + timestamp_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (serverNameBuilder_ == null) { + result.serverName_ = serverName_; + } else { + result.serverName_ = serverNameBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.timestamp_ = timestamp_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()) return this; + if (other.hasServerName()) { + mergeServerName(other.getServerName()); + } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (hasServerName()) { + if (!getServerName().isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_; + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public boolean hasServerName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName() { + if (serverNameBuilder_ == null) { + return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_; + } else { + return serverNameBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public Builder setServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + serverName_ = value; + onChanged(); + } else { + serverNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public Builder setServerName( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serverNameBuilder_ == null) { + serverName_ = builderForValue.build(); + onChanged(); + } else { + serverNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public Builder mergeServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverNameBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + serverName_ != null && + serverName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + serverName_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.newBuilder(serverName_).mergeFrom(value).buildPartial(); + } else { + serverName_ = value; + } + onChanged(); + } else { + serverNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public Builder clearServerName() { + if (serverNameBuilder_ == null) { + serverName_ = null; + onChanged(); + } else { + serverNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServerNameBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getServerNameFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() { + if (serverNameBuilder_ != null) { + return serverNameBuilder_.getMessageOrBuilder(); + } else { + return serverName_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_; + } + } + /** + * optional .hbase.pb.ServerName server_name = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServerNameFieldBuilder() { + if (serverNameBuilder_ == null) { + serverNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + getServerName(), + getParentForChildren(), + isClean()); + serverName_ = null; + } + return serverNameBuilder_; + } + + private long timestamp_ ; + /** + * optional uint64 timestamp = 2; + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 timestamp = 2; + */ + public long getTimestamp() { + return timestamp_; + } + /** + * optional uint64 timestamp = 2; + */ + public Builder setTimestamp(long value) { + bitField0_ |= 0x00000002; + timestamp_ = value; + onChanged(); + return this; + } + /** + * optional uint64 timestamp = 2; + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = 0L; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.ServerTimestamp) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ServerTimestamp) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public ServerTimestamp parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new ServerTimestamp(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface TableServerTimestampOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.TableServerTimestamp) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional .hbase.pb.TableName table_name = 1; + */ + boolean hasTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + java.util.List + getServerTimestampList(); + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index); + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + int getServerTimestampCount(); + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + java.util.List + getServerTimestampOrBuilderList(); + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder( + int index); + } + /** + *
+   **
+   *  TableServerTimestamp keeps last WAL roll time per Region Server & Table
+   *  Each table have different last WAL roll time stamps across cluster, on every RS
+   * 
+ * + * Protobuf type {@code hbase.pb.TableServerTimestamp} + */ + public static final class TableServerTimestamp extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.TableServerTimestamp) + TableServerTimestampOrBuilder { + // Use TableServerTimestamp.newBuilder() to construct. + private TableServerTimestamp(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private TableServerTimestamp() { + serverTimestamp_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TableServerTimestamp( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = tableName_.toBuilder(); + } + tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(tableName_); + tableName_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + serverTimestamp_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000002; + } + serverTimestamp_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + serverTimestamp_ = java.util.Collections.unmodifiableList(serverTimestamp_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder.class); + } + + private int bitField0_; + public static final int TABLE_NAME_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + + public static final int SERVER_TIMESTAMP_FIELD_NUMBER = 2; + private java.util.List serverTimestamp_; + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public java.util.List getServerTimestampList() { + return serverTimestamp_; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public java.util.List + getServerTimestampOrBuilderList() { + return serverTimestamp_; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public int getServerTimestampCount() { + return serverTimestamp_.size(); + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index) { + return serverTimestamp_.get(index); + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder( + int index) { + return serverTimestamp_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasTableName()) { + if (!getTableName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getServerTimestampCount(); i++) { + if (!getServerTimestamp(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getTableName()); + } + for (int i = 0; i < serverTimestamp_.size(); i++) { + output.writeMessage(2, serverTimestamp_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getTableName()); + } + for (int i = 0; i < serverTimestamp_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(2, serverTimestamp_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) obj; + + boolean result = true; + result = result && (hasTableName() == other.hasTableName()); + if (hasTableName()) { + result = result && getTableName() + .equals(other.getTableName()); + } + result = result && getServerTimestampList() + .equals(other.getServerTimestampList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTableName()) { + hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; + hash = (53 * hash) + getTableName().hashCode(); + } + if (getServerTimestampCount() > 0) { + hash = (37 * hash) + SERVER_TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + getServerTimestampList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     *  TableServerTimestamp keeps last WAL roll time per Region Server & Table
+     *  Each table have different last WAL roll time stamps across cluster, on every RS
+     * 
+ * + * Protobuf type {@code hbase.pb.TableServerTimestamp} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.TableServerTimestamp) + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getTableNameFieldBuilder(); + getServerTimestampFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (tableNameBuilder_ == null) { + tableName_ = null; + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (serverTimestampBuilder_ == null) { + serverTimestamp_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + } else { + serverTimestampBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (tableNameBuilder_ == null) { + result.tableName_ = tableName_; + } else { + result.tableName_ = tableNameBuilder_.build(); + } + if (serverTimestampBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { + serverTimestamp_ = java.util.Collections.unmodifiableList(serverTimestamp_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.serverTimestamp_ = serverTimestamp_; + } else { + result.serverTimestamp_ = serverTimestampBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance()) return this; + if (other.hasTableName()) { + mergeTableName(other.getTableName()); + } + if (serverTimestampBuilder_ == null) { + if (!other.serverTimestamp_.isEmpty()) { + if (serverTimestamp_.isEmpty()) { + serverTimestamp_ = other.serverTimestamp_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureServerTimestampIsMutable(); + serverTimestamp_.addAll(other.serverTimestamp_); + } + onChanged(); + } + } else { + if (!other.serverTimestamp_.isEmpty()) { + if (serverTimestampBuilder_.isEmpty()) { + serverTimestampBuilder_.dispose(); + serverTimestampBuilder_ = null; + serverTimestamp_ = other.serverTimestamp_; + bitField0_ = (bitField0_ & ~0x00000002); + serverTimestampBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getServerTimestampFieldBuilder() : null; + } else { + serverTimestampBuilder_.addAllMessages(other.serverTimestamp_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (hasTableName()) { + if (!getTableName().isInitialized()) { + return false; + } + } + for (int i = 0; i < getServerTimestampCount(); i++) { + if (!getServerTimestamp(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() { + if (tableNameBuilder_ == null) { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } else { + return tableNameBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tableName_ = value; + onChanged(); + } else { + tableNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableNameBuilder_ == null) { + tableName_ = builderForValue.build(); + onChanged(); + } else { + tableNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + tableName_ != null && + tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) { + tableName_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial(); + } else { + tableName_ = value; + } + onChanged(); + } else { + tableNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder clearTableName() { + if (tableNameBuilder_ == null) { + tableName_ = null; + onChanged(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getTableNameFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + if (tableNameBuilder_ != null) { + return tableNameBuilder_.getMessageOrBuilder(); + } else { + return tableName_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> + getTableNameFieldBuilder() { + if (tableNameBuilder_ == null) { + tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>( + getTableName(), + getParentForChildren(), + isClean()); + tableName_ = null; + } + return tableNameBuilder_; + } + + private java.util.List serverTimestamp_ = + java.util.Collections.emptyList(); + private void ensureServerTimestampIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + serverTimestamp_ = new java.util.ArrayList(serverTimestamp_); + bitField0_ |= 0x00000002; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> serverTimestampBuilder_; + + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public java.util.List getServerTimestampList() { + if (serverTimestampBuilder_ == null) { + return java.util.Collections.unmodifiableList(serverTimestamp_); + } else { + return serverTimestampBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public int getServerTimestampCount() { + if (serverTimestampBuilder_ == null) { + return serverTimestamp_.size(); + } else { + return serverTimestampBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index) { + if (serverTimestampBuilder_ == null) { + return serverTimestamp_.get(index); + } else { + return serverTimestampBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder setServerTimestamp( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) { + if (serverTimestampBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServerTimestampIsMutable(); + serverTimestamp_.set(index, value); + onChanged(); + } else { + serverTimestampBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder setServerTimestamp( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) { + if (serverTimestampBuilder_ == null) { + ensureServerTimestampIsMutable(); + serverTimestamp_.set(index, builderForValue.build()); + onChanged(); + } else { + serverTimestampBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder addServerTimestamp(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) { + if (serverTimestampBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServerTimestampIsMutable(); + serverTimestamp_.add(value); + onChanged(); + } else { + serverTimestampBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder addServerTimestamp( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) { + if (serverTimestampBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServerTimestampIsMutable(); + serverTimestamp_.add(index, value); + onChanged(); + } else { + serverTimestampBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder addServerTimestamp( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) { + if (serverTimestampBuilder_ == null) { + ensureServerTimestampIsMutable(); + serverTimestamp_.add(builderForValue.build()); + onChanged(); + } else { + serverTimestampBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder addServerTimestamp( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) { + if (serverTimestampBuilder_ == null) { + ensureServerTimestampIsMutable(); + serverTimestamp_.add(index, builderForValue.build()); + onChanged(); + } else { + serverTimestampBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder addAllServerTimestamp( + java.lang.Iterable values) { + if (serverTimestampBuilder_ == null) { + ensureServerTimestampIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, serverTimestamp_); + onChanged(); + } else { + serverTimestampBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder clearServerTimestamp() { + if (serverTimestampBuilder_ == null) { + serverTimestamp_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + serverTimestampBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public Builder removeServerTimestamp(int index) { + if (serverTimestampBuilder_ == null) { + ensureServerTimestampIsMutable(); + serverTimestamp_.remove(index); + onChanged(); + } else { + serverTimestampBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder getServerTimestampBuilder( + int index) { + return getServerTimestampFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder( + int index) { + if (serverTimestampBuilder_ == null) { + return serverTimestamp_.get(index); } else { + return serverTimestampBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public java.util.List + getServerTimestampOrBuilderList() { + if (serverTimestampBuilder_ != null) { + return serverTimestampBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(serverTimestamp_); + } + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder addServerTimestampBuilder() { + return getServerTimestampFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder addServerTimestampBuilder( + int index) { + return getServerTimestampFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerTimestamp server_timestamp = 2; + */ + public java.util.List + getServerTimestampBuilderList() { + return getServerTimestampFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> + getServerTimestampFieldBuilder() { + if (serverTimestampBuilder_ == null) { + serverTimestampBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder>( + serverTimestamp_, + ((bitField0_ & 0x00000002) == 0x00000002), + getParentForChildren(), + isClean()); + serverTimestamp_ = null; + } + return serverTimestampBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.TableServerTimestamp) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.TableServerTimestamp) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public TableServerTimestamp parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new TableServerTimestamp(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface BackupImageOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.BackupImage) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional string backup_id = 1; + */ + boolean hasBackupId(); + /** + * optional string backup_id = 1; + */ + java.lang.String getBackupId(); + /** + * optional string backup_id = 1; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes(); + + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + boolean hasBackupType(); + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType(); + + /** + * optional string backup_root_dir = 3; + */ + boolean hasBackupRootDir(); + /** + * optional string backup_root_dir = 3; + */ + java.lang.String getBackupRootDir(); + /** + * optional string backup_root_dir = 3; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes(); + + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + java.util.List + getTableListList(); + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableList(int index); + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + int getTableListCount(); + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + java.util.List + getTableListOrBuilderList(); + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder( + int index); + + /** + * optional uint64 start_ts = 5; + */ + boolean hasStartTs(); + /** + * optional uint64 start_ts = 5; + */ + long getStartTs(); + + /** + * optional uint64 complete_ts = 6; + */ + boolean hasCompleteTs(); + /** + * optional uint64 complete_ts = 6; + */ + long getCompleteTs(); + + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + java.util.List + getAncestorsList(); + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getAncestors(int index); + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + int getAncestorsCount(); + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + java.util.List + getAncestorsOrBuilderList(); + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder( + int index); + + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + java.util.List + getTstMapList(); + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getTstMap(int index); + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + int getTstMapCount(); + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + java.util.List + getTstMapOrBuilderList(); + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder getTstMapOrBuilder( + int index); + } + /** + *
+   **
+   * Structure keeps relevant info for backup restore session
+   * 
+ * + * Protobuf type {@code hbase.pb.BackupImage} + */ + public static final class BackupImage extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.BackupImage) + BackupImageOrBuilder { + // Use BackupImage.newBuilder() to construct. + private BackupImage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private BackupImage() { + backupId_ = ""; + backupType_ = 0; + backupRootDir_ = ""; + tableList_ = java.util.Collections.emptyList(); + startTs_ = 0L; + completeTs_ = 0L; + ancestors_ = java.util.Collections.emptyList(); + tstMap_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private BackupImage( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + backupId_ = bs; + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(2, rawValue); + } else { + bitField0_ |= 0x00000002; + backupType_ = rawValue; + } + break; + } + case 26: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000004; + backupRootDir_ = bs; + break; + } + case 34: { + if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + tableList_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000008; + } + tableList_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry)); + break; + } + case 40: { + bitField0_ |= 0x00000008; + startTs_ = input.readUInt64(); + break; + } + case 48: { + bitField0_ |= 0x00000010; + completeTs_ = input.readUInt64(); + break; + } + case 58: { + if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) { + ancestors_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000040; + } + ancestors_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry)); + break; + } + case 66: { + if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) { + tstMap_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000080; + } + tstMap_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + tableList_ = java.util.Collections.unmodifiableList(tableList_); + } + if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) { + ancestors_ = java.util.Collections.unmodifiableList(ancestors_); + } + if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) { + tstMap_ = java.util.Collections.unmodifiableList(tstMap_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder.class); + } + + private int bitField0_; + public static final int BACKUP_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object backupId_; + /** + * optional string backup_id = 1; + */ + public boolean hasBackupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional string backup_id = 1; + */ + public java.lang.String getBackupId() { + java.lang.Object ref = backupId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupId_ = s; + } + return s; + } + } + /** + * optional string backup_id = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes() { + java.lang.Object ref = backupId_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupId_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int BACKUP_TYPE_FIELD_NUMBER = 2; + private int backupType_; + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public boolean hasBackupType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(backupType_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.FULL : result; + } + + public static final int BACKUP_ROOT_DIR_FIELD_NUMBER = 3; + private volatile java.lang.Object backupRootDir_; + /** + * optional string backup_root_dir = 3; + */ + public boolean hasBackupRootDir() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional string backup_root_dir = 3; + */ + public java.lang.String getBackupRootDir() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupRootDir_ = s; + } + return s; + } + } + /** + * optional string backup_root_dir = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupRootDir_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int TABLE_LIST_FIELD_NUMBER = 4; + private java.util.List tableList_; + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public java.util.List getTableListList() { + return tableList_; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public java.util.List + getTableListOrBuilderList() { + return tableList_; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public int getTableListCount() { + return tableList_.size(); + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableList(int index) { + return tableList_.get(index); + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder( + int index) { + return tableList_.get(index); + } + + public static final int START_TS_FIELD_NUMBER = 5; + private long startTs_; + /** + * optional uint64 start_ts = 5; + */ + public boolean hasStartTs() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 start_ts = 5; + */ + public long getStartTs() { + return startTs_; + } + + public static final int COMPLETE_TS_FIELD_NUMBER = 6; + private long completeTs_; + /** + * optional uint64 complete_ts = 6; + */ + public boolean hasCompleteTs() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional uint64 complete_ts = 6; + */ + public long getCompleteTs() { + return completeTs_; + } + + public static final int ANCESTORS_FIELD_NUMBER = 7; + private java.util.List ancestors_; + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public java.util.List getAncestorsList() { + return ancestors_; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public java.util.List + getAncestorsOrBuilderList() { + return ancestors_; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public int getAncestorsCount() { + return ancestors_.size(); + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) { + return ancestors_.get(index); + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder( + int index) { + return ancestors_.get(index); + } + + public static final int TST_MAP_FIELD_NUMBER = 8; + private java.util.List tstMap_; + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public java.util.List getTstMapList() { + return tstMap_; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public java.util.List + getTstMapOrBuilderList() { + return tstMap_; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public int getTstMapCount() { + return tstMap_.size(); + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getTstMap(int index) { + return tstMap_.get(index); + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder getTstMapOrBuilder( + int index) { + return tstMap_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getTableListCount(); i++) { + if (!getTableList(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getAncestorsCount(); i++) { + if (!getAncestors(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getTstMapCount(); i++) { + if (!getTstMap(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, backupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, backupType_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, backupRootDir_); + } + for (int i = 0; i < tableList_.size(); i++) { + output.writeMessage(4, tableList_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(5, startTs_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeUInt64(6, completeTs_); + } + for (int i = 0; i < ancestors_.size(); i++) { + output.writeMessage(7, ancestors_.get(i)); + } + for (int i = 0; i < tstMap_.size(); i++) { + output.writeMessage(8, tstMap_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, backupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeEnumSize(2, backupType_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, backupRootDir_); + } + for (int i = 0; i < tableList_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(4, tableList_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(5, startTs_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(6, completeTs_); + } + for (int i = 0; i < ancestors_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(7, ancestors_.get(i)); + } + for (int i = 0; i < tstMap_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(8, tstMap_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage) obj; + + boolean result = true; + result = result && (hasBackupId() == other.hasBackupId()); + if (hasBackupId()) { + result = result && getBackupId() + .equals(other.getBackupId()); + } + result = result && (hasBackupType() == other.hasBackupType()); + if (hasBackupType()) { + result = result && backupType_ == other.backupType_; + } + result = result && (hasBackupRootDir() == other.hasBackupRootDir()); + if (hasBackupRootDir()) { + result = result && getBackupRootDir() + .equals(other.getBackupRootDir()); + } + result = result && getTableListList() + .equals(other.getTableListList()); + result = result && (hasStartTs() == other.hasStartTs()); + if (hasStartTs()) { + result = result && (getStartTs() + == other.getStartTs()); + } + result = result && (hasCompleteTs() == other.hasCompleteTs()); + if (hasCompleteTs()) { + result = result && (getCompleteTs() + == other.getCompleteTs()); + } + result = result && getAncestorsList() + .equals(other.getAncestorsList()); + result = result && getTstMapList() + .equals(other.getTstMapList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasBackupId()) { + hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER; + hash = (53 * hash) + getBackupId().hashCode(); + } + if (hasBackupType()) { + hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER; + hash = (53 * hash) + backupType_; + } + if (hasBackupRootDir()) { + hash = (37 * hash) + BACKUP_ROOT_DIR_FIELD_NUMBER; + hash = (53 * hash) + getBackupRootDir().hashCode(); + } + if (getTableListCount() > 0) { + hash = (37 * hash) + TABLE_LIST_FIELD_NUMBER; + hash = (53 * hash) + getTableListList().hashCode(); + } + if (hasStartTs()) { + hash = (37 * hash) + START_TS_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getStartTs()); + } + if (hasCompleteTs()) { + hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getCompleteTs()); + } + if (getAncestorsCount() > 0) { + hash = (37 * hash) + ANCESTORS_FIELD_NUMBER; + hash = (53 * hash) + getAncestorsList().hashCode(); + } + if (getTstMapCount() > 0) { + hash = (37 * hash) + TST_MAP_FIELD_NUMBER; + hash = (53 * hash) + getTstMapList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     * Structure keeps relevant info for backup restore session
+     * 
+ * + * Protobuf type {@code hbase.pb.BackupImage} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.BackupImage) + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getTableListFieldBuilder(); + getAncestorsFieldBuilder(); + getTstMapFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + backupId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + backupType_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + backupRootDir_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + if (tableListBuilder_ == null) { + tableList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + } else { + tableListBuilder_.clear(); + } + startTs_ = 0L; + bitField0_ = (bitField0_ & ~0x00000010); + completeTs_ = 0L; + bitField0_ = (bitField0_ & ~0x00000020); + if (ancestorsBuilder_ == null) { + ancestors_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + } else { + ancestorsBuilder_.clear(); + } + if (tstMapBuilder_ == null) { + tstMap_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); + } else { + tstMapBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.backupId_ = backupId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.backupType_ = backupType_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.backupRootDir_ = backupRootDir_; + if (tableListBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { + tableList_ = java.util.Collections.unmodifiableList(tableList_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.tableList_ = tableList_; + } else { + result.tableList_ = tableListBuilder_.build(); + } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000008; + } + result.startTs_ = startTs_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000010; + } + result.completeTs_ = completeTs_; + if (ancestorsBuilder_ == null) { + if (((bitField0_ & 0x00000040) == 0x00000040)) { + ancestors_ = java.util.Collections.unmodifiableList(ancestors_); + bitField0_ = (bitField0_ & ~0x00000040); + } + result.ancestors_ = ancestors_; + } else { + result.ancestors_ = ancestorsBuilder_.build(); + } + if (tstMapBuilder_ == null) { + if (((bitField0_ & 0x00000080) == 0x00000080)) { + tstMap_ = java.util.Collections.unmodifiableList(tstMap_); + bitField0_ = (bitField0_ & ~0x00000080); + } + result.tstMap_ = tstMap_; + } else { + result.tstMap_ = tstMapBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()) return this; + if (other.hasBackupId()) { + bitField0_ |= 0x00000001; + backupId_ = other.backupId_; + onChanged(); + } + if (other.hasBackupType()) { + setBackupType(other.getBackupType()); + } + if (other.hasBackupRootDir()) { + bitField0_ |= 0x00000004; + backupRootDir_ = other.backupRootDir_; + onChanged(); + } + if (tableListBuilder_ == null) { + if (!other.tableList_.isEmpty()) { + if (tableList_.isEmpty()) { + tableList_ = other.tableList_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureTableListIsMutable(); + tableList_.addAll(other.tableList_); + } + onChanged(); + } + } else { + if (!other.tableList_.isEmpty()) { + if (tableListBuilder_.isEmpty()) { + tableListBuilder_.dispose(); + tableListBuilder_ = null; + tableList_ = other.tableList_; + bitField0_ = (bitField0_ & ~0x00000008); + tableListBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getTableListFieldBuilder() : null; + } else { + tableListBuilder_.addAllMessages(other.tableList_); + } + } + } + if (other.hasStartTs()) { + setStartTs(other.getStartTs()); + } + if (other.hasCompleteTs()) { + setCompleteTs(other.getCompleteTs()); + } + if (ancestorsBuilder_ == null) { + if (!other.ancestors_.isEmpty()) { + if (ancestors_.isEmpty()) { + ancestors_ = other.ancestors_; + bitField0_ = (bitField0_ & ~0x00000040); + } else { + ensureAncestorsIsMutable(); + ancestors_.addAll(other.ancestors_); + } + onChanged(); + } + } else { + if (!other.ancestors_.isEmpty()) { + if (ancestorsBuilder_.isEmpty()) { + ancestorsBuilder_.dispose(); + ancestorsBuilder_ = null; + ancestors_ = other.ancestors_; + bitField0_ = (bitField0_ & ~0x00000040); + ancestorsBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getAncestorsFieldBuilder() : null; + } else { + ancestorsBuilder_.addAllMessages(other.ancestors_); + } + } + } + if (tstMapBuilder_ == null) { + if (!other.tstMap_.isEmpty()) { + if (tstMap_.isEmpty()) { + tstMap_ = other.tstMap_; + bitField0_ = (bitField0_ & ~0x00000080); + } else { + ensureTstMapIsMutable(); + tstMap_.addAll(other.tstMap_); + } + onChanged(); + } + } else { + if (!other.tstMap_.isEmpty()) { + if (tstMapBuilder_.isEmpty()) { + tstMapBuilder_.dispose(); + tstMapBuilder_ = null; + tstMap_ = other.tstMap_; + bitField0_ = (bitField0_ & ~0x00000080); + tstMapBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getTstMapFieldBuilder() : null; + } else { + tstMapBuilder_.addAllMessages(other.tstMap_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getTableListCount(); i++) { + if (!getTableList(i).isInitialized()) { + return false; + } + } + for (int i = 0; i < getAncestorsCount(); i++) { + if (!getAncestors(i).isInitialized()) { + return false; + } + } + for (int i = 0; i < getTstMapCount(); i++) { + if (!getTstMap(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.lang.Object backupId_ = ""; + /** + * optional string backup_id = 1; + */ + public boolean hasBackupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional string backup_id = 1; + */ + public java.lang.String getBackupId() { + java.lang.Object ref = backupId_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string backup_id = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes() { + java.lang.Object ref = backupId_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupId_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string backup_id = 1; + */ + public Builder setBackupId( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + backupId_ = value; + onChanged(); + return this; + } + /** + * optional string backup_id = 1; + */ + public Builder clearBackupId() { + bitField0_ = (bitField0_ & ~0x00000001); + backupId_ = getDefaultInstance().getBackupId(); + onChanged(); + return this; + } + /** + * optional string backup_id = 1; + */ + public Builder setBackupIdBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + backupId_ = value; + onChanged(); + return this; + } + + private int backupType_ = 0; + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public boolean hasBackupType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(backupType_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.FULL : result; + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public Builder setBackupType(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + backupType_ = value.getNumber(); + onChanged(); + return this; + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public Builder clearBackupType() { + bitField0_ = (bitField0_ & ~0x00000002); + backupType_ = 0; + onChanged(); + return this; + } + + private java.lang.Object backupRootDir_ = ""; + /** + * optional string backup_root_dir = 3; + */ + public boolean hasBackupRootDir() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional string backup_root_dir = 3; + */ + public java.lang.String getBackupRootDir() { + java.lang.Object ref = backupRootDir_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupRootDir_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string backup_root_dir = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupRootDir_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string backup_root_dir = 3; + */ + public Builder setBackupRootDir( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + backupRootDir_ = value; + onChanged(); + return this; + } + /** + * optional string backup_root_dir = 3; + */ + public Builder clearBackupRootDir() { + bitField0_ = (bitField0_ & ~0x00000004); + backupRootDir_ = getDefaultInstance().getBackupRootDir(); + onChanged(); + return this; + } + /** + * optional string backup_root_dir = 3; + */ + public Builder setBackupRootDirBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + backupRootDir_ = value; + onChanged(); + return this; + } + + private java.util.List tableList_ = + java.util.Collections.emptyList(); + private void ensureTableListIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + tableList_ = new java.util.ArrayList(tableList_); + bitField0_ |= 0x00000008; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableListBuilder_; + + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public java.util.List getTableListList() { + if (tableListBuilder_ == null) { + return java.util.Collections.unmodifiableList(tableList_); + } else { + return tableListBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public int getTableListCount() { + if (tableListBuilder_ == null) { + return tableList_.size(); + } else { + return tableListBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableList(int index) { + if (tableListBuilder_ == null) { + return tableList_.get(index); + } else { + return tableListBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder setTableList( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableListIsMutable(); + tableList_.set(index, value); + onChanged(); + } else { + tableListBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder setTableList( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableListBuilder_ == null) { + ensureTableListIsMutable(); + tableList_.set(index, builderForValue.build()); + onChanged(); + } else { + tableListBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder addTableList(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableListIsMutable(); + tableList_.add(value); + onChanged(); + } else { + tableListBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder addTableList( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableListIsMutable(); + tableList_.add(index, value); + onChanged(); + } else { + tableListBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder addTableList( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableListBuilder_ == null) { + ensureTableListIsMutable(); + tableList_.add(builderForValue.build()); + onChanged(); + } else { + tableListBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder addTableList( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableListBuilder_ == null) { + ensureTableListIsMutable(); + tableList_.add(index, builderForValue.build()); + onChanged(); + } else { + tableListBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder addAllTableList( + java.lang.Iterable values) { + if (tableListBuilder_ == null) { + ensureTableListIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, tableList_); + onChanged(); + } else { + tableListBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder clearTableList() { + if (tableListBuilder_ == null) { + tableList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + } else { + tableListBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public Builder removeTableList(int index) { + if (tableListBuilder_ == null) { + ensureTableListIsMutable(); + tableList_.remove(index); + onChanged(); + } else { + tableListBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableListBuilder( + int index) { + return getTableListFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder( + int index) { + if (tableListBuilder_ == null) { + return tableList_.get(index); } else { + return tableListBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public java.util.List + getTableListOrBuilderList() { + if (tableListBuilder_ != null) { + return tableListBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(tableList_); + } + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder() { + return getTableListFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder( + int index) { + return getTableListFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableName table_list = 4; + */ + public java.util.List + getTableListBuilderList() { + return getTableListFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> + getTableListFieldBuilder() { + if (tableListBuilder_ == null) { + tableListBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>( + tableList_, + ((bitField0_ & 0x00000008) == 0x00000008), + getParentForChildren(), + isClean()); + tableList_ = null; + } + return tableListBuilder_; + } + + private long startTs_ ; + /** + * optional uint64 start_ts = 5; + */ + public boolean hasStartTs() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional uint64 start_ts = 5; + */ + public long getStartTs() { + return startTs_; + } + /** + * optional uint64 start_ts = 5; + */ + public Builder setStartTs(long value) { + bitField0_ |= 0x00000010; + startTs_ = value; + onChanged(); + return this; + } + /** + * optional uint64 start_ts = 5; + */ + public Builder clearStartTs() { + bitField0_ = (bitField0_ & ~0x00000010); + startTs_ = 0L; + onChanged(); + return this; + } + + private long completeTs_ ; + /** + * optional uint64 complete_ts = 6; + */ + public boolean hasCompleteTs() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional uint64 complete_ts = 6; + */ + public long getCompleteTs() { + return completeTs_; + } + /** + * optional uint64 complete_ts = 6; + */ + public Builder setCompleteTs(long value) { + bitField0_ |= 0x00000020; + completeTs_ = value; + onChanged(); + return this; + } + /** + * optional uint64 complete_ts = 6; + */ + public Builder clearCompleteTs() { + bitField0_ = (bitField0_ & ~0x00000020); + completeTs_ = 0L; + onChanged(); + return this; + } + + private java.util.List ancestors_ = + java.util.Collections.emptyList(); + private void ensureAncestorsIsMutable() { + if (!((bitField0_ & 0x00000040) == 0x00000040)) { + ancestors_ = new java.util.ArrayList(ancestors_); + bitField0_ |= 0x00000040; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder> ancestorsBuilder_; + + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public java.util.List getAncestorsList() { + if (ancestorsBuilder_ == null) { + return java.util.Collections.unmodifiableList(ancestors_); + } else { + return ancestorsBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public int getAncestorsCount() { + if (ancestorsBuilder_ == null) { + return ancestors_.size(); + } else { + return ancestorsBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) { + if (ancestorsBuilder_ == null) { + return ancestors_.get(index); + } else { + return ancestorsBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder setAncestors( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage value) { + if (ancestorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureAncestorsIsMutable(); + ancestors_.set(index, value); + onChanged(); + } else { + ancestorsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder setAncestors( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) { + if (ancestorsBuilder_ == null) { + ensureAncestorsIsMutable(); + ancestors_.set(index, builderForValue.build()); + onChanged(); + } else { + ancestorsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder addAncestors(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage value) { + if (ancestorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureAncestorsIsMutable(); + ancestors_.add(value); + onChanged(); + } else { + ancestorsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder addAncestors( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage value) { + if (ancestorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureAncestorsIsMutable(); + ancestors_.add(index, value); + onChanged(); + } else { + ancestorsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder addAncestors( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) { + if (ancestorsBuilder_ == null) { + ensureAncestorsIsMutable(); + ancestors_.add(builderForValue.build()); + onChanged(); + } else { + ancestorsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder addAncestors( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) { + if (ancestorsBuilder_ == null) { + ensureAncestorsIsMutable(); + ancestors_.add(index, builderForValue.build()); + onChanged(); + } else { + ancestorsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder addAllAncestors( + java.lang.Iterable values) { + if (ancestorsBuilder_ == null) { + ensureAncestorsIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, ancestors_); + onChanged(); + } else { + ancestorsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder clearAncestors() { + if (ancestorsBuilder_ == null) { + ancestors_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + onChanged(); + } else { + ancestorsBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public Builder removeAncestors(int index) { + if (ancestorsBuilder_ == null) { + ensureAncestorsIsMutable(); + ancestors_.remove(index); + onChanged(); + } else { + ancestorsBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder getAncestorsBuilder( + int index) { + return getAncestorsFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder( + int index) { + if (ancestorsBuilder_ == null) { + return ancestors_.get(index); } else { + return ancestorsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public java.util.List + getAncestorsOrBuilderList() { + if (ancestorsBuilder_ != null) { + return ancestorsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(ancestors_); + } + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder() { + return getAncestorsFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()); + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder( + int index) { + return getAncestorsFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()); + } + /** + * repeated .hbase.pb.BackupImage ancestors = 7; + */ + public java.util.List + getAncestorsBuilderList() { + return getAncestorsFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder> + getAncestorsFieldBuilder() { + if (ancestorsBuilder_ == null) { + ancestorsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder>( + ancestors_, + ((bitField0_ & 0x00000040) == 0x00000040), + getParentForChildren(), + isClean()); + ancestors_ = null; + } + return ancestorsBuilder_; + } + + private java.util.List tstMap_ = + java.util.Collections.emptyList(); + private void ensureTstMapIsMutable() { + if (!((bitField0_ & 0x00000080) == 0x00000080)) { + tstMap_ = new java.util.ArrayList(tstMap_); + bitField0_ |= 0x00000080; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder> tstMapBuilder_; + + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public java.util.List getTstMapList() { + if (tstMapBuilder_ == null) { + return java.util.Collections.unmodifiableList(tstMap_); + } else { + return tstMapBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public int getTstMapCount() { + if (tstMapBuilder_ == null) { + return tstMap_.size(); + } else { + return tstMapBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getTstMap(int index) { + if (tstMapBuilder_ == null) { + return tstMap_.get(index); + } else { + return tstMapBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder setTstMap( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp value) { + if (tstMapBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTstMapIsMutable(); + tstMap_.set(index, value); + onChanged(); + } else { + tstMapBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder setTstMap( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder builderForValue) { + if (tstMapBuilder_ == null) { + ensureTstMapIsMutable(); + tstMap_.set(index, builderForValue.build()); + onChanged(); + } else { + tstMapBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder addTstMap(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp value) { + if (tstMapBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTstMapIsMutable(); + tstMap_.add(value); + onChanged(); + } else { + tstMapBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder addTstMap( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp value) { + if (tstMapBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTstMapIsMutable(); + tstMap_.add(index, value); + onChanged(); + } else { + tstMapBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder addTstMap( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder builderForValue) { + if (tstMapBuilder_ == null) { + ensureTstMapIsMutable(); + tstMap_.add(builderForValue.build()); + onChanged(); + } else { + tstMapBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder addTstMap( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder builderForValue) { + if (tstMapBuilder_ == null) { + ensureTstMapIsMutable(); + tstMap_.add(index, builderForValue.build()); + onChanged(); + } else { + tstMapBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder addAllTstMap( + java.lang.Iterable values) { + if (tstMapBuilder_ == null) { + ensureTstMapIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, tstMap_); + onChanged(); + } else { + tstMapBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder clearTstMap() { + if (tstMapBuilder_ == null) { + tstMap_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + } else { + tstMapBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public Builder removeTstMap(int index) { + if (tstMapBuilder_ == null) { + ensureTstMapIsMutable(); + tstMap_.remove(index); + onChanged(); + } else { + tstMapBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder getTstMapBuilder( + int index) { + return getTstMapFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder getTstMapOrBuilder( + int index) { + if (tstMapBuilder_ == null) { + return tstMap_.get(index); } else { + return tstMapBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public java.util.List + getTstMapOrBuilderList() { + if (tstMapBuilder_ != null) { + return tstMapBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(tstMap_); + } + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder addTstMapBuilder() { + return getTstMapFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder addTstMapBuilder( + int index) { + return getTstMapFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableServerTimestamp tst_map = 8; + */ + public java.util.List + getTstMapBuilderList() { + return getTstMapFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder> + getTstMapFieldBuilder() { + if (tstMapBuilder_ == null) { + tstMapBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder>( + tstMap_, + ((bitField0_ & 0x00000080) == 0x00000080), + getParentForChildren(), + isClean()); + tstMap_ = null; + } + return tstMapBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.BackupImage) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.BackupImage) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public BackupImage parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new BackupImage(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface BackupTableInfoOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.BackupTableInfo) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional .hbase.pb.TableName table_name = 1; + */ + boolean hasTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + /** + * optional string snapshot_name = 2; + */ + boolean hasSnapshotName(); + /** + * optional string snapshot_name = 2; + */ + java.lang.String getSnapshotName(); + /** + * optional string snapshot_name = 2; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getSnapshotNameBytes(); + } + /** + *
+   **
+   * Internal structure used during backup
+   * 
+ * + * Protobuf type {@code hbase.pb.BackupTableInfo} + */ + public static final class BackupTableInfo extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.BackupTableInfo) + BackupTableInfoOrBuilder { + // Use BackupTableInfo.newBuilder() to construct. + private BackupTableInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private BackupTableInfo() { + snapshotName_ = ""; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private BackupTableInfo( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = tableName_.toBuilder(); + } + tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(tableName_); + tableName_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000002; + snapshotName_ = bs; + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupTableInfo_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupTableInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder.class); + } + + private int bitField0_; + public static final int TABLE_NAME_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + + public static final int SNAPSHOT_NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object snapshotName_; + /** + * optional string snapshot_name = 2; + */ + public boolean hasSnapshotName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional string snapshot_name = 2; + */ + public java.lang.String getSnapshotName() { + java.lang.Object ref = snapshotName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + snapshotName_ = s; + } + return s; + } + } + /** + * optional string snapshot_name = 2; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getSnapshotNameBytes() { + java.lang.Object ref = snapshotName_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + snapshotName_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasTableName()) { + if (!getTableName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getTableName()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, snapshotName_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getTableName()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, snapshotName_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo) obj; + + boolean result = true; + result = result && (hasTableName() == other.hasTableName()); + if (hasTableName()) { + result = result && getTableName() + .equals(other.getTableName()); + } + result = result && (hasSnapshotName() == other.hasSnapshotName()); + if (hasSnapshotName()) { + result = result && getSnapshotName() + .equals(other.getSnapshotName()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTableName()) { + hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; + hash = (53 * hash) + getTableName().hashCode(); + } + if (hasSnapshotName()) { + hash = (37 * hash) + SNAPSHOT_NAME_FIELD_NUMBER; + hash = (53 * hash) + getSnapshotName().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     * Internal structure used during backup
+     * 
+ * + * Protobuf type {@code hbase.pb.BackupTableInfo} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.BackupTableInfo) + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupTableInfo_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupTableInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getTableNameFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (tableNameBuilder_ == null) { + tableName_ = null; + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + snapshotName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupTableInfo_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (tableNameBuilder_ == null) { + result.tableName_ = tableName_; + } else { + result.tableName_ = tableNameBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.snapshotName_ = snapshotName_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.getDefaultInstance()) return this; + if (other.hasTableName()) { + mergeTableName(other.getTableName()); + } + if (other.hasSnapshotName()) { + bitField0_ |= 0x00000002; + snapshotName_ = other.snapshotName_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (hasTableName()) { + if (!getTableName().isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() { + if (tableNameBuilder_ == null) { + return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } else { + return tableNameBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tableName_ = value; + onChanged(); + } else { + tableNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableNameBuilder_ == null) { + tableName_ = builderForValue.build(); + onChanged(); + } else { + tableNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + tableName_ != null && + tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) { + tableName_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial(); + } else { + tableName_ = value; + } + onChanged(); + } else { + tableNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder clearTableName() { + if (tableNameBuilder_ == null) { + tableName_ = null; + onChanged(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getTableNameFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + if (tableNameBuilder_ != null) { + return tableNameBuilder_.getMessageOrBuilder(); + } else { + return tableName_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_; + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> + getTableNameFieldBuilder() { + if (tableNameBuilder_ == null) { + tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>( + getTableName(), + getParentForChildren(), + isClean()); + tableName_ = null; + } + return tableNameBuilder_; + } + + private java.lang.Object snapshotName_ = ""; + /** + * optional string snapshot_name = 2; + */ + public boolean hasSnapshotName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional string snapshot_name = 2; + */ + public java.lang.String getSnapshotName() { + java.lang.Object ref = snapshotName_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + snapshotName_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string snapshot_name = 2; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getSnapshotNameBytes() { + java.lang.Object ref = snapshotName_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + snapshotName_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string snapshot_name = 2; + */ + public Builder setSnapshotName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + snapshotName_ = value; + onChanged(); + return this; + } + /** + * optional string snapshot_name = 2; + */ + public Builder clearSnapshotName() { + bitField0_ = (bitField0_ & ~0x00000002); + snapshotName_ = getDefaultInstance().getSnapshotName(); + onChanged(); + return this; + } + /** + * optional string snapshot_name = 2; + */ + public Builder setSnapshotNameBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + snapshotName_ = value; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.BackupTableInfo) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.BackupTableInfo) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public BackupTableInfo parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new BackupTableInfo(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface BackupInfoOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.BackupInfo) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional string backup_id = 1; + */ + boolean hasBackupId(); + /** + * optional string backup_id = 1; + */ + java.lang.String getBackupId(); + /** + * optional string backup_id = 1; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes(); + + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + boolean hasBackupType(); + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType(); + + /** + * optional string backup_root_dir = 3; + */ + boolean hasBackupRootDir(); + /** + * optional string backup_root_dir = 3; + */ + java.lang.String getBackupRootDir(); + /** + * optional string backup_root_dir = 3; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes(); + + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + boolean hasBackupState(); + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState getBackupState(); + + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + boolean hasBackupPhase(); + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase getBackupPhase(); + + /** + * optional string failed_message = 6; + */ + boolean hasFailedMessage(); + /** + * optional string failed_message = 6; + */ + java.lang.String getFailedMessage(); + /** + * optional string failed_message = 6; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getFailedMessageBytes(); + + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + java.util.List + getBackupTableInfoList(); + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getBackupTableInfo(int index); + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + int getBackupTableInfoCount(); + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + java.util.List + getBackupTableInfoOrBuilderList(); + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder getBackupTableInfoOrBuilder( + int index); + + /** + * optional uint64 start_ts = 8; + */ + boolean hasStartTs(); + /** + * optional uint64 start_ts = 8; + */ + long getStartTs(); + + /** + * optional uint64 complete_ts = 9; + */ + boolean hasCompleteTs(); + /** + * optional uint64 complete_ts = 9; + */ + long getCompleteTs(); + + /** + * optional uint32 progress = 10; + */ + boolean hasProgress(); + /** + * optional uint32 progress = 10; + */ + int getProgress(); + + /** + * optional uint32 workers_number = 11; + */ + boolean hasWorkersNumber(); + /** + * optional uint32 workers_number = 11; + */ + int getWorkersNumber(); + + /** + * optional uint64 bandwidth = 12; + */ + boolean hasBandwidth(); + /** + * optional uint64 bandwidth = 12; + */ + long getBandwidth(); + } + /** + *
+   **
+   * Backup session information
+   * 
+ * + * Protobuf type {@code hbase.pb.BackupInfo} + */ + public static final class BackupInfo extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.BackupInfo) + BackupInfoOrBuilder { + // Use BackupInfo.newBuilder() to construct. + private BackupInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private BackupInfo() { + backupId_ = ""; + backupType_ = 0; + backupRootDir_ = ""; + backupState_ = 0; + backupPhase_ = 0; + failedMessage_ = ""; + backupTableInfo_ = java.util.Collections.emptyList(); + startTs_ = 0L; + completeTs_ = 0L; + progress_ = 0; + workersNumber_ = 0; + bandwidth_ = 0L; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private BackupInfo( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + backupId_ = bs; + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(2, rawValue); + } else { + bitField0_ |= 0x00000002; + backupType_ = rawValue; + } + break; + } + case 26: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000004; + backupRootDir_ = bs; + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState value = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(4, rawValue); + } else { + bitField0_ |= 0x00000008; + backupState_ = rawValue; + } + break; + } + case 40: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase value = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(5, rawValue); + } else { + bitField0_ |= 0x00000010; + backupPhase_ = rawValue; + } + break; + } + case 50: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000020; + failedMessage_ = bs; + break; + } + case 58: { + if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) { + backupTableInfo_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000040; + } + backupTableInfo_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.PARSER, extensionRegistry)); + break; + } + case 64: { + bitField0_ |= 0x00000040; + startTs_ = input.readUInt64(); + break; + } + case 72: { + bitField0_ |= 0x00000080; + completeTs_ = input.readUInt64(); + break; + } + case 80: { + bitField0_ |= 0x00000100; + progress_ = input.readUInt32(); + break; + } + case 88: { + bitField0_ |= 0x00000200; + workersNumber_ = input.readUInt32(); + break; + } + case 96: { + bitField0_ |= 0x00000400; + bandwidth_ = input.readUInt64(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) { + backupTableInfo_ = java.util.Collections.unmodifiableList(backupTableInfo_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupInfo_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder.class); + } + + /** + *
+     **
+     * Backup session states
+     * 
+ * + * Protobuf enum {@code hbase.pb.BackupInfo.BackupState} + */ + public enum BackupState + implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum { + /** + * RUNNING = 0; + */ + RUNNING(0), + /** + * COMPLETE = 1; + */ + COMPLETE(1), + /** + * FAILED = 2; + */ + FAILED(2), + /** + * CANCELLED = 3; + */ + CANCELLED(3), + ; + + /** + * RUNNING = 0; + */ + public static final int RUNNING_VALUE = 0; + /** + * COMPLETE = 1; + */ + public static final int COMPLETE_VALUE = 1; + /** + * FAILED = 2; + */ + public static final int FAILED_VALUE = 2; + /** + * CANCELLED = 3; + */ + public static final int CANCELLED_VALUE = 3; + + + public final int getNumber() { + return value; + } + + /** + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static BackupState valueOf(int value) { + return forNumber(value); + } + + public static BackupState forNumber(int value) { + switch (value) { + case 0: return RUNNING; + case 1: return COMPLETE; + case 2: return FAILED; + case 3: return CANCELLED; + default: return null; + } + } + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap< + BackupState> internalValueMap = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() { + public BackupState findValueByNumber(int number) { + return BackupState.forNumber(number); + } + }; + + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(ordinal()); + } + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.getDescriptor().getEnumTypes().get(0); + } + + private static final BackupState[] VALUES = values(); + + public static BackupState valueOf( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private BackupState(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:hbase.pb.BackupInfo.BackupState) + } + + /** + *
+     **
+     * Phases of a backup session in RUNNING state
+     * 
+ * + * Protobuf enum {@code hbase.pb.BackupInfo.BackupPhase} + */ + public enum BackupPhase + implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum { + /** + * REQUEST = 0; + */ + REQUEST(0), + /** + * SNAPSHOT = 1; + */ + SNAPSHOT(1), + /** + * PREPARE_INCREMENTAL = 2; + */ + PREPARE_INCREMENTAL(2), + /** + * SNAPSHOTCOPY = 3; + */ + SNAPSHOTCOPY(3), + /** + * INCREMENTAL_COPY = 4; + */ + INCREMENTAL_COPY(4), + /** + * STORE_MANIFEST = 5; + */ + STORE_MANIFEST(5), + ; + + /** + * REQUEST = 0; + */ + public static final int REQUEST_VALUE = 0; + /** + * SNAPSHOT = 1; + */ + public static final int SNAPSHOT_VALUE = 1; + /** + * PREPARE_INCREMENTAL = 2; + */ + public static final int PREPARE_INCREMENTAL_VALUE = 2; + /** + * SNAPSHOTCOPY = 3; + */ + public static final int SNAPSHOTCOPY_VALUE = 3; + /** + * INCREMENTAL_COPY = 4; + */ + public static final int INCREMENTAL_COPY_VALUE = 4; + /** + * STORE_MANIFEST = 5; + */ + public static final int STORE_MANIFEST_VALUE = 5; + + + public final int getNumber() { + return value; + } + + /** + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static BackupPhase valueOf(int value) { + return forNumber(value); + } + + public static BackupPhase forNumber(int value) { + switch (value) { + case 0: return REQUEST; + case 1: return SNAPSHOT; + case 2: return PREPARE_INCREMENTAL; + case 3: return SNAPSHOTCOPY; + case 4: return INCREMENTAL_COPY; + case 5: return STORE_MANIFEST; + default: return null; + } + } + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap< + BackupPhase> internalValueMap = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() { + public BackupPhase findValueByNumber(int number) { + return BackupPhase.forNumber(number); + } + }; + + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(ordinal()); + } + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.getDescriptor().getEnumTypes().get(1); + } + + private static final BackupPhase[] VALUES = values(); + + public static BackupPhase valueOf( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private BackupPhase(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:hbase.pb.BackupInfo.BackupPhase) + } + + private int bitField0_; + public static final int BACKUP_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object backupId_; + /** + * optional string backup_id = 1; + */ + public boolean hasBackupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional string backup_id = 1; + */ + public java.lang.String getBackupId() { + java.lang.Object ref = backupId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupId_ = s; + } + return s; + } + } + /** + * optional string backup_id = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes() { + java.lang.Object ref = backupId_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupId_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int BACKUP_TYPE_FIELD_NUMBER = 2; + private int backupType_; + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public boolean hasBackupType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(backupType_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.FULL : result; + } + + public static final int BACKUP_ROOT_DIR_FIELD_NUMBER = 3; + private volatile java.lang.Object backupRootDir_; + /** + * optional string backup_root_dir = 3; + */ + public boolean hasBackupRootDir() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional string backup_root_dir = 3; + */ + public java.lang.String getBackupRootDir() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupRootDir_ = s; + } + return s; + } + } + /** + * optional string backup_root_dir = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupRootDir_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int BACKUP_STATE_FIELD_NUMBER = 4; + private int backupState_; + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public boolean hasBackupState() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState getBackupState() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState.valueOf(backupState_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState.RUNNING : result; + } + + public static final int BACKUP_PHASE_FIELD_NUMBER = 5; + private int backupPhase_; + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public boolean hasBackupPhase() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase getBackupPhase() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase.valueOf(backupPhase_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase.REQUEST : result; + } + + public static final int FAILED_MESSAGE_FIELD_NUMBER = 6; + private volatile java.lang.Object failedMessage_; + /** + * optional string failed_message = 6; + */ + public boolean hasFailedMessage() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional string failed_message = 6; + */ + public java.lang.String getFailedMessage() { + java.lang.Object ref = failedMessage_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + failedMessage_ = s; + } + return s; + } + } + /** + * optional string failed_message = 6; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getFailedMessageBytes() { + java.lang.Object ref = failedMessage_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + failedMessage_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int BACKUP_TABLE_INFO_FIELD_NUMBER = 7; + private java.util.List backupTableInfo_; + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public java.util.List getBackupTableInfoList() { + return backupTableInfo_; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public java.util.List + getBackupTableInfoOrBuilderList() { + return backupTableInfo_; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public int getBackupTableInfoCount() { + return backupTableInfo_.size(); + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getBackupTableInfo(int index) { + return backupTableInfo_.get(index); + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder getBackupTableInfoOrBuilder( + int index) { + return backupTableInfo_.get(index); + } + + public static final int START_TS_FIELD_NUMBER = 8; + private long startTs_; + /** + * optional uint64 start_ts = 8; + */ + public boolean hasStartTs() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional uint64 start_ts = 8; + */ + public long getStartTs() { + return startTs_; + } + + public static final int COMPLETE_TS_FIELD_NUMBER = 9; + private long completeTs_; + /** + * optional uint64 complete_ts = 9; + */ + public boolean hasCompleteTs() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional uint64 complete_ts = 9; + */ + public long getCompleteTs() { + return completeTs_; + } + + public static final int PROGRESS_FIELD_NUMBER = 10; + private int progress_; + /** + * optional uint32 progress = 10; + */ + public boolean hasProgress() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional uint32 progress = 10; + */ + public int getProgress() { + return progress_; + } + + public static final int WORKERS_NUMBER_FIELD_NUMBER = 11; + private int workersNumber_; + /** + * optional uint32 workers_number = 11; + */ + public boolean hasWorkersNumber() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional uint32 workers_number = 11; + */ + public int getWorkersNumber() { + return workersNumber_; + } + + public static final int BANDWIDTH_FIELD_NUMBER = 12; + private long bandwidth_; + /** + * optional uint64 bandwidth = 12; + */ + public boolean hasBandwidth() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional uint64 bandwidth = 12; + */ + public long getBandwidth() { + return bandwidth_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getBackupTableInfoCount(); i++) { + if (!getBackupTableInfo(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, backupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, backupType_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, backupRootDir_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, backupState_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeEnum(5, backupPhase_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 6, failedMessage_); + } + for (int i = 0; i < backupTableInfo_.size(); i++) { + output.writeMessage(7, backupTableInfo_.get(i)); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeUInt64(8, startTs_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeUInt64(9, completeTs_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt32(10, progress_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeUInt32(11, workersNumber_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeUInt64(12, bandwidth_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, backupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeEnumSize(2, backupType_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, backupRootDir_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeEnumSize(4, backupState_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeEnumSize(5, backupPhase_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(6, failedMessage_); + } + for (int i = 0; i < backupTableInfo_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(7, backupTableInfo_.get(i)); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(8, startTs_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(9, completeTs_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt32Size(10, progress_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt32Size(11, workersNumber_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeUInt64Size(12, bandwidth_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo) obj; + + boolean result = true; + result = result && (hasBackupId() == other.hasBackupId()); + if (hasBackupId()) { + result = result && getBackupId() + .equals(other.getBackupId()); + } + result = result && (hasBackupType() == other.hasBackupType()); + if (hasBackupType()) { + result = result && backupType_ == other.backupType_; + } + result = result && (hasBackupRootDir() == other.hasBackupRootDir()); + if (hasBackupRootDir()) { + result = result && getBackupRootDir() + .equals(other.getBackupRootDir()); + } + result = result && (hasBackupState() == other.hasBackupState()); + if (hasBackupState()) { + result = result && backupState_ == other.backupState_; + } + result = result && (hasBackupPhase() == other.hasBackupPhase()); + if (hasBackupPhase()) { + result = result && backupPhase_ == other.backupPhase_; + } + result = result && (hasFailedMessage() == other.hasFailedMessage()); + if (hasFailedMessage()) { + result = result && getFailedMessage() + .equals(other.getFailedMessage()); + } + result = result && getBackupTableInfoList() + .equals(other.getBackupTableInfoList()); + result = result && (hasStartTs() == other.hasStartTs()); + if (hasStartTs()) { + result = result && (getStartTs() + == other.getStartTs()); + } + result = result && (hasCompleteTs() == other.hasCompleteTs()); + if (hasCompleteTs()) { + result = result && (getCompleteTs() + == other.getCompleteTs()); + } + result = result && (hasProgress() == other.hasProgress()); + if (hasProgress()) { + result = result && (getProgress() + == other.getProgress()); + } + result = result && (hasWorkersNumber() == other.hasWorkersNumber()); + if (hasWorkersNumber()) { + result = result && (getWorkersNumber() + == other.getWorkersNumber()); + } + result = result && (hasBandwidth() == other.hasBandwidth()); + if (hasBandwidth()) { + result = result && (getBandwidth() + == other.getBandwidth()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasBackupId()) { + hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER; + hash = (53 * hash) + getBackupId().hashCode(); + } + if (hasBackupType()) { + hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER; + hash = (53 * hash) + backupType_; + } + if (hasBackupRootDir()) { + hash = (37 * hash) + BACKUP_ROOT_DIR_FIELD_NUMBER; + hash = (53 * hash) + getBackupRootDir().hashCode(); + } + if (hasBackupState()) { + hash = (37 * hash) + BACKUP_STATE_FIELD_NUMBER; + hash = (53 * hash) + backupState_; + } + if (hasBackupPhase()) { + hash = (37 * hash) + BACKUP_PHASE_FIELD_NUMBER; + hash = (53 * hash) + backupPhase_; + } + if (hasFailedMessage()) { + hash = (37 * hash) + FAILED_MESSAGE_FIELD_NUMBER; + hash = (53 * hash) + getFailedMessage().hashCode(); + } + if (getBackupTableInfoCount() > 0) { + hash = (37 * hash) + BACKUP_TABLE_INFO_FIELD_NUMBER; + hash = (53 * hash) + getBackupTableInfoList().hashCode(); + } + if (hasStartTs()) { + hash = (37 * hash) + START_TS_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getStartTs()); + } + if (hasCompleteTs()) { + hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getCompleteTs()); + } + if (hasProgress()) { + hash = (37 * hash) + PROGRESS_FIELD_NUMBER; + hash = (53 * hash) + getProgress(); + } + if (hasWorkersNumber()) { + hash = (37 * hash) + WORKERS_NUMBER_FIELD_NUMBER; + hash = (53 * hash) + getWorkersNumber(); + } + if (hasBandwidth()) { + hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getBandwidth()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+     **
+     * Backup session information
+     * 
+ * + * Protobuf type {@code hbase.pb.BackupInfo} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.BackupInfo) + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfoOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupInfo_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getBackupTableInfoFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + backupId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + backupType_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + backupRootDir_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + backupState_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + backupPhase_ = 0; + bitField0_ = (bitField0_ & ~0x00000010); + failedMessage_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + if (backupTableInfoBuilder_ == null) { + backupTableInfo_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + } else { + backupTableInfoBuilder_.clear(); + } + startTs_ = 0L; + bitField0_ = (bitField0_ & ~0x00000080); + completeTs_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); + progress_ = 0; + bitField0_ = (bitField0_ & ~0x00000200); + workersNumber_ = 0; + bitField0_ = (bitField0_ & ~0x00000400); + bandwidth_ = 0L; + bitField0_ = (bitField0_ & ~0x00000800); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupInfo_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.backupId_ = backupId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.backupType_ = backupType_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.backupRootDir_ = backupRootDir_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.backupState_ = backupState_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.backupPhase_ = backupPhase_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.failedMessage_ = failedMessage_; + if (backupTableInfoBuilder_ == null) { + if (((bitField0_ & 0x00000040) == 0x00000040)) { + backupTableInfo_ = java.util.Collections.unmodifiableList(backupTableInfo_); + bitField0_ = (bitField0_ & ~0x00000040); + } + result.backupTableInfo_ = backupTableInfo_; + } else { + result.backupTableInfo_ = backupTableInfoBuilder_.build(); + } + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000040; + } + result.startTs_ = startTs_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000080; + } + result.completeTs_ = completeTs_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000100; + } + result.progress_ = progress_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000200; + } + result.workersNumber_ = workersNumber_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000400; + } + result.bandwidth_ = bandwidth_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.getDefaultInstance()) return this; + if (other.hasBackupId()) { + bitField0_ |= 0x00000001; + backupId_ = other.backupId_; + onChanged(); + } + if (other.hasBackupType()) { + setBackupType(other.getBackupType()); + } + if (other.hasBackupRootDir()) { + bitField0_ |= 0x00000004; + backupRootDir_ = other.backupRootDir_; + onChanged(); + } + if (other.hasBackupState()) { + setBackupState(other.getBackupState()); + } + if (other.hasBackupPhase()) { + setBackupPhase(other.getBackupPhase()); + } + if (other.hasFailedMessage()) { + bitField0_ |= 0x00000020; + failedMessage_ = other.failedMessage_; + onChanged(); + } + if (backupTableInfoBuilder_ == null) { + if (!other.backupTableInfo_.isEmpty()) { + if (backupTableInfo_.isEmpty()) { + backupTableInfo_ = other.backupTableInfo_; + bitField0_ = (bitField0_ & ~0x00000040); + } else { + ensureBackupTableInfoIsMutable(); + backupTableInfo_.addAll(other.backupTableInfo_); + } + onChanged(); + } + } else { + if (!other.backupTableInfo_.isEmpty()) { + if (backupTableInfoBuilder_.isEmpty()) { + backupTableInfoBuilder_.dispose(); + backupTableInfoBuilder_ = null; + backupTableInfo_ = other.backupTableInfo_; + bitField0_ = (bitField0_ & ~0x00000040); + backupTableInfoBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getBackupTableInfoFieldBuilder() : null; + } else { + backupTableInfoBuilder_.addAllMessages(other.backupTableInfo_); + } + } + } + if (other.hasStartTs()) { + setStartTs(other.getStartTs()); + } + if (other.hasCompleteTs()) { + setCompleteTs(other.getCompleteTs()); + } + if (other.hasProgress()) { + setProgress(other.getProgress()); + } + if (other.hasWorkersNumber()) { + setWorkersNumber(other.getWorkersNumber()); + } + if (other.hasBandwidth()) { + setBandwidth(other.getBandwidth()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getBackupTableInfoCount(); i++) { + if (!getBackupTableInfo(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.lang.Object backupId_ = ""; + /** + * optional string backup_id = 1; + */ + public boolean hasBackupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional string backup_id = 1; + */ + public java.lang.String getBackupId() { + java.lang.Object ref = backupId_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string backup_id = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupIdBytes() { + java.lang.Object ref = backupId_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupId_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string backup_id = 1; + */ + public Builder setBackupId( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + backupId_ = value; + onChanged(); + return this; + } + /** + * optional string backup_id = 1; + */ + public Builder clearBackupId() { + bitField0_ = (bitField0_ & ~0x00000001); + backupId_ = getDefaultInstance().getBackupId(); + onChanged(); + return this; + } + /** + * optional string backup_id = 1; + */ + public Builder setBackupIdBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + backupId_ = value; + onChanged(); + return this; + } + + private int backupType_ = 0; + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public boolean hasBackupType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(backupType_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.FULL : result; + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public Builder setBackupType(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + backupType_ = value.getNumber(); + onChanged(); + return this; + } + /** + * optional .hbase.pb.BackupType backup_type = 2; + */ + public Builder clearBackupType() { + bitField0_ = (bitField0_ & ~0x00000002); + backupType_ = 0; + onChanged(); + return this; + } + + private java.lang.Object backupRootDir_ = ""; + /** + * optional string backup_root_dir = 3; + */ + public boolean hasBackupRootDir() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional string backup_root_dir = 3; + */ + public java.lang.String getBackupRootDir() { + java.lang.Object ref = backupRootDir_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + backupRootDir_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string backup_root_dir = 3; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getBackupRootDirBytes() { + java.lang.Object ref = backupRootDir_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + backupRootDir_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string backup_root_dir = 3; + */ + public Builder setBackupRootDir( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + backupRootDir_ = value; + onChanged(); + return this; + } + /** + * optional string backup_root_dir = 3; + */ + public Builder clearBackupRootDir() { + bitField0_ = (bitField0_ & ~0x00000004); + backupRootDir_ = getDefaultInstance().getBackupRootDir(); + onChanged(); + return this; + } + /** + * optional string backup_root_dir = 3; + */ + public Builder setBackupRootDirBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + backupRootDir_ = value; + onChanged(); + return this; + } + + private int backupState_ = 0; + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public boolean hasBackupState() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState getBackupState() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState.valueOf(backupState_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState.RUNNING : result; + } + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public Builder setBackupState(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupState value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + backupState_ = value.getNumber(); + onChanged(); + return this; + } + /** + * optional .hbase.pb.BackupInfo.BackupState backup_state = 4; + */ + public Builder clearBackupState() { + bitField0_ = (bitField0_ & ~0x00000008); + backupState_ = 0; + onChanged(); + return this; + } + + private int backupPhase_ = 0; + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public boolean hasBackupPhase() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase getBackupPhase() { + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase result = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase.valueOf(backupPhase_); + return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase.REQUEST : result; + } + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public Builder setBackupPhase(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.BackupPhase value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + backupPhase_ = value.getNumber(); + onChanged(); + return this; + } + /** + * optional .hbase.pb.BackupInfo.BackupPhase backup_phase = 5; + */ + public Builder clearBackupPhase() { + bitField0_ = (bitField0_ & ~0x00000010); + backupPhase_ = 0; + onChanged(); + return this; + } + + private java.lang.Object failedMessage_ = ""; + /** + * optional string failed_message = 6; + */ + public boolean hasFailedMessage() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional string failed_message = 6; + */ + public java.lang.String getFailedMessage() { + java.lang.Object ref = failedMessage_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + failedMessage_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string failed_message = 6; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getFailedMessageBytes() { + java.lang.Object ref = failedMessage_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + failedMessage_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * optional string failed_message = 6; + */ + public Builder setFailedMessage( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + failedMessage_ = value; + onChanged(); + return this; + } + /** + * optional string failed_message = 6; + */ + public Builder clearFailedMessage() { + bitField0_ = (bitField0_ & ~0x00000020); + failedMessage_ = getDefaultInstance().getFailedMessage(); + onChanged(); + return this; + } + /** + * optional string failed_message = 6; + */ + public Builder setFailedMessageBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + failedMessage_ = value; + onChanged(); + return this; + } + + private java.util.List backupTableInfo_ = + java.util.Collections.emptyList(); + private void ensureBackupTableInfoIsMutable() { + if (!((bitField0_ & 0x00000040) == 0x00000040)) { + backupTableInfo_ = new java.util.ArrayList(backupTableInfo_); + bitField0_ |= 0x00000040; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder> backupTableInfoBuilder_; + + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public java.util.List getBackupTableInfoList() { + if (backupTableInfoBuilder_ == null) { + return java.util.Collections.unmodifiableList(backupTableInfo_); + } else { + return backupTableInfoBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public int getBackupTableInfoCount() { + if (backupTableInfoBuilder_ == null) { + return backupTableInfo_.size(); + } else { + return backupTableInfoBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo getBackupTableInfo(int index) { + if (backupTableInfoBuilder_ == null) { + return backupTableInfo_.get(index); + } else { + return backupTableInfoBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder setBackupTableInfo( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo value) { + if (backupTableInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBackupTableInfoIsMutable(); + backupTableInfo_.set(index, value); + onChanged(); + } else { + backupTableInfoBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder setBackupTableInfo( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder builderForValue) { + if (backupTableInfoBuilder_ == null) { + ensureBackupTableInfoIsMutable(); + backupTableInfo_.set(index, builderForValue.build()); + onChanged(); + } else { + backupTableInfoBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder addBackupTableInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo value) { + if (backupTableInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBackupTableInfoIsMutable(); + backupTableInfo_.add(value); + onChanged(); + } else { + backupTableInfoBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder addBackupTableInfo( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo value) { + if (backupTableInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBackupTableInfoIsMutable(); + backupTableInfo_.add(index, value); + onChanged(); + } else { + backupTableInfoBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder addBackupTableInfo( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder builderForValue) { + if (backupTableInfoBuilder_ == null) { + ensureBackupTableInfoIsMutable(); + backupTableInfo_.add(builderForValue.build()); + onChanged(); + } else { + backupTableInfoBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder addBackupTableInfo( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder builderForValue) { + if (backupTableInfoBuilder_ == null) { + ensureBackupTableInfoIsMutable(); + backupTableInfo_.add(index, builderForValue.build()); + onChanged(); + } else { + backupTableInfoBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder addAllBackupTableInfo( + java.lang.Iterable values) { + if (backupTableInfoBuilder_ == null) { + ensureBackupTableInfoIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, backupTableInfo_); + onChanged(); + } else { + backupTableInfoBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder clearBackupTableInfo() { + if (backupTableInfoBuilder_ == null) { + backupTableInfo_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + onChanged(); + } else { + backupTableInfoBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public Builder removeBackupTableInfo(int index) { + if (backupTableInfoBuilder_ == null) { + ensureBackupTableInfoIsMutable(); + backupTableInfo_.remove(index); + onChanged(); + } else { + backupTableInfoBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder getBackupTableInfoBuilder( + int index) { + return getBackupTableInfoFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder getBackupTableInfoOrBuilder( + int index) { + if (backupTableInfoBuilder_ == null) { + return backupTableInfo_.get(index); } else { + return backupTableInfoBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public java.util.List + getBackupTableInfoOrBuilderList() { + if (backupTableInfoBuilder_ != null) { + return backupTableInfoBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(backupTableInfo_); + } + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder addBackupTableInfoBuilder() { + return getBackupTableInfoFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.getDefaultInstance()); + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder addBackupTableInfoBuilder( + int index) { + return getBackupTableInfoFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.getDefaultInstance()); + } + /** + * repeated .hbase.pb.BackupTableInfo backup_table_info = 7; + */ + public java.util.List + getBackupTableInfoBuilderList() { + return getBackupTableInfoFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder> + getBackupTableInfoFieldBuilder() { + if (backupTableInfoBuilder_ == null) { + backupTableInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupTableInfoOrBuilder>( + backupTableInfo_, + ((bitField0_ & 0x00000040) == 0x00000040), + getParentForChildren(), + isClean()); + backupTableInfo_ = null; + } + return backupTableInfoBuilder_; + } + + private long startTs_ ; + /** + * optional uint64 start_ts = 8; + */ + public boolean hasStartTs() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional uint64 start_ts = 8; + */ + public long getStartTs() { + return startTs_; + } + /** + * optional uint64 start_ts = 8; + */ + public Builder setStartTs(long value) { + bitField0_ |= 0x00000080; + startTs_ = value; + onChanged(); + return this; + } + /** + * optional uint64 start_ts = 8; + */ + public Builder clearStartTs() { + bitField0_ = (bitField0_ & ~0x00000080); + startTs_ = 0L; + onChanged(); + return this; + } + + private long completeTs_ ; + /** + * optional uint64 complete_ts = 9; + */ + public boolean hasCompleteTs() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional uint64 complete_ts = 9; + */ + public long getCompleteTs() { + return completeTs_; + } + /** + * optional uint64 complete_ts = 9; + */ + public Builder setCompleteTs(long value) { + bitField0_ |= 0x00000100; + completeTs_ = value; + onChanged(); + return this; + } + /** + * optional uint64 complete_ts = 9; + */ + public Builder clearCompleteTs() { + bitField0_ = (bitField0_ & ~0x00000100); + completeTs_ = 0L; + onChanged(); + return this; + } + + private int progress_ ; + /** + * optional uint32 progress = 10; + */ + public boolean hasProgress() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional uint32 progress = 10; + */ + public int getProgress() { + return progress_; + } + /** + * optional uint32 progress = 10; + */ + public Builder setProgress(int value) { + bitField0_ |= 0x00000200; + progress_ = value; + onChanged(); + return this; + } + /** + * optional uint32 progress = 10; + */ + public Builder clearProgress() { + bitField0_ = (bitField0_ & ~0x00000200); + progress_ = 0; + onChanged(); + return this; + } + + private int workersNumber_ ; + /** + * optional uint32 workers_number = 11; + */ + public boolean hasWorkersNumber() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional uint32 workers_number = 11; + */ + public int getWorkersNumber() { + return workersNumber_; + } + /** + * optional uint32 workers_number = 11; + */ + public Builder setWorkersNumber(int value) { + bitField0_ |= 0x00000400; + workersNumber_ = value; + onChanged(); + return this; + } + /** + * optional uint32 workers_number = 11; + */ + public Builder clearWorkersNumber() { + bitField0_ = (bitField0_ & ~0x00000400); + workersNumber_ = 0; + onChanged(); + return this; + } + + private long bandwidth_ ; + /** + * optional uint64 bandwidth = 12; + */ + public boolean hasBandwidth() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + /** + * optional uint64 bandwidth = 12; + */ + public long getBandwidth() { + return bandwidth_; + } + /** + * optional uint64 bandwidth = 12; + */ + public Builder setBandwidth(long value) { + bitField0_ |= 0x00000800; + bandwidth_ = value; + onChanged(); + return this; + } + /** + * optional uint64 bandwidth = 12; + */ + public Builder clearBandwidth() { + bitField0_ = (bitField0_ & ~0x00000800); + bandwidth_ = 0L; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.BackupInfo) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.BackupInfo) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public BackupInfo parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new BackupInfo(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ServerTimestamp_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_TableServerTimestamp_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_BackupImage_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_BackupImage_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_BackupTableInfo_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_BackupTableInfo_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_BackupInfo_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_BackupInfo_fieldAccessorTable; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\014Backup.proto\022\010hbase.pb\032\013HBase.proto\"O\n" + + "\017ServerTimestamp\022)\n\013server_name\030\001 \001(\0132\024." + + "hbase.pb.ServerName\022\021\n\ttimestamp\030\002 \001(\004\"t" + + "\n\024TableServerTimestamp\022\'\n\ntable_name\030\001 \001" + + "(\0132\023.hbase.pb.TableName\0223\n\020server_timest" + + "amp\030\002 \003(\0132\031.hbase.pb.ServerTimestamp\"\217\002\n" + + "\013BackupImage\022\021\n\tbackup_id\030\001 \001(\t\022)\n\013backu" + + "p_type\030\002 \001(\0162\024.hbase.pb.BackupType\022\027\n\017ba" + + "ckup_root_dir\030\003 \001(\t\022\'\n\ntable_list\030\004 \003(\0132" + + "\023.hbase.pb.TableName\022\020\n\010start_ts\030\005 \001(\004\022\023", + "\n\013complete_ts\030\006 \001(\004\022(\n\tancestors\030\007 \003(\0132\025" + + ".hbase.pb.BackupImage\022/\n\007tst_map\030\010 \003(\0132\036" + + ".hbase.pb.TableServerTimestamp\"Q\n\017Backup" + + "TableInfo\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb" + + ".TableName\022\025\n\rsnapshot_name\030\002 \001(\t\"\311\004\n\nBa" + + "ckupInfo\022\021\n\tbackup_id\030\001 \001(\t\022)\n\013backup_ty" + + "pe\030\002 \001(\0162\024.hbase.pb.BackupType\022\027\n\017backup" + + "_root_dir\030\003 \001(\t\0226\n\014backup_state\030\004 \001(\0162 ." + + "hbase.pb.BackupInfo.BackupState\0226\n\014backu" + + "p_phase\030\005 \001(\0162 .hbase.pb.BackupInfo.Back", + "upPhase\022\026\n\016failed_message\030\006 \001(\t\0224\n\021backu" + + "p_table_info\030\007 \003(\0132\031.hbase.pb.BackupTabl" + + "eInfo\022\020\n\010start_ts\030\010 \001(\004\022\023\n\013complete_ts\030\t" + + " \001(\004\022\020\n\010progress\030\n \001(\r\022\026\n\016workers_number" + + "\030\013 \001(\r\022\021\n\tbandwidth\030\014 \001(\004\"C\n\013BackupState" + + "\022\013\n\007RUNNING\020\000\022\014\n\010COMPLETE\020\001\022\n\n\006FAILED\020\002\022" + + "\r\n\tCANCELLED\020\003\"}\n\013BackupPhase\022\013\n\007REQUEST" + + "\020\000\022\014\n\010SNAPSHOT\020\001\022\027\n\023PREPARE_INCREMENTAL\020" + + "\002\022\020\n\014SNAPSHOTCOPY\020\003\022\024\n\020INCREMENTAL_COPY\020" + + "\004\022\022\n\016STORE_MANIFEST\020\005*\'\n\nBackupType\022\010\n\004F", + "ULL\020\000\022\017\n\013INCREMENTAL\020\001BI\n1org.apache.had" + + "oop.hbase.shaded.protobuf.generatedB\014Bac" + + "kupProtosH\001\210\001\001\240\001\001" + }; + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(), + }, assigner); + internal_static_hbase_pb_ServerTimestamp_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_ServerTimestamp_descriptor, + new java.lang.String[] { "ServerName", "Timestamp", }); + internal_static_hbase_pb_TableServerTimestamp_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_TableServerTimestamp_descriptor, + new java.lang.String[] { "TableName", "ServerTimestamp", }); + internal_static_hbase_pb_BackupImage_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_hbase_pb_BackupImage_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_BackupImage_descriptor, + new java.lang.String[] { "BackupId", "BackupType", "BackupRootDir", "TableList", "StartTs", "CompleteTs", "Ancestors", "TstMap", }); + internal_static_hbase_pb_BackupTableInfo_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_hbase_pb_BackupTableInfo_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_BackupTableInfo_descriptor, + new java.lang.String[] { "TableName", "SnapshotName", }); + internal_static_hbase_pb_BackupInfo_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_hbase_pb_BackupInfo_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_BackupInfo_descriptor, + new java.lang.String[] { "BackupId", "BackupType", "BackupRootDir", "BackupState", "BackupPhase", "FailedMessage", "BackupTableInfo", "StartTs", "CompleteTs", "Progress", "WorkersNumber", "Bandwidth", }); + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java index 52b0ce56999..bb6b40e92fe 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java @@ -42223,7 +42223,7 @@ public final class MasterProtos { * required .hbase.pb.SnapshotDescription snapshot = 1; */ private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< - org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> getSnapshotFieldBuilder() { if (snapshotBuilder_ == null) { snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< @@ -70510,7 +70510,7 @@ public final class MasterProtos { /** *
-       ** Get a run of the CleanerChore
+       ** Get a run of the CleanerChore 
        * 
* * rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse); @@ -72424,7 +72424,7 @@ public final class MasterProtos { /** *
-     ** Get a run of the CleanerChore
+     ** Get a run of the CleanerChore 
      * 
* * rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse); @@ -76186,32 +76186,32 @@ public final class MasterProtos { internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_RunCleanerChoreRequest_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_RunCleanerChoreRequest_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_RunCleanerChoreResponse_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_RunCleanerChoreResponse_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_SetCleanerChoreRunningRequest_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_SetCleanerChoreRunningRequest_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_SetCleanerChoreRunningResponse_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_SetCleanerChoreRunningResponse_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_IsCleanerChoreEnabledRequest_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_IsCleanerChoreEnabledRequest_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_IsCleanerChoreEnabledResponse_descriptor; - private static final + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_IsCleanerChoreEnabledResponse_fieldAccessorTable; private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto new file mode 100644 index 00000000000..241d42a8b70 --- /dev/null +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless optional by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// This file contains Backup manifest +package hbase.pb; + +option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated"; +option java_outer_classname = "BackupProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +import "HBase.proto"; + +/** + * Backup type enum: FULL or INCREMENTAL + */ + +enum BackupType { + FULL = 0; + INCREMENTAL = 1; +} +/** + * ServerTimestamp keeps last WAL roll time per Region Server + */ +message ServerTimestamp { + optional ServerName server_name = 1; + optional uint64 timestamp = 2; +} + +/** + * TableServerTimestamp keeps last WAL roll time per Region Server & Table + * Each table have different last WAL roll time stamps across cluster, on every RS + */ +message TableServerTimestamp { + optional TableName table_name = 1; + repeated ServerTimestamp server_timestamp = 2; +} + +/** + * Structure keeps relevant info for backup restore session + */ +message BackupImage { + optional string backup_id = 1; + optional BackupType backup_type = 2; + optional string backup_root_dir = 3; + repeated TableName table_list = 4; + optional uint64 start_ts = 5; + optional uint64 complete_ts = 6; + repeated BackupImage ancestors = 7; + repeated TableServerTimestamp tst_map = 8; + +} + +/** + * Internal structure used during backup + */ +message BackupTableInfo { + optional TableName table_name = 1; + optional string snapshot_name = 2; +} + +/** + * Backup session information + */ +message BackupInfo { + optional string backup_id = 1; + optional BackupType backup_type = 2; + optional string backup_root_dir = 3; + optional BackupState backup_state = 4; + optional BackupPhase backup_phase = 5; + optional string failed_message = 6; + repeated BackupTableInfo backup_table_info = 7; + optional uint64 start_ts = 8; + optional uint64 complete_ts = 9; + optional uint32 progress = 10; + optional uint32 workers_number = 11; + optional uint64 bandwidth = 12; + + /** + * Backup session states + */ + enum BackupState { + RUNNING = 0; + COMPLETE = 1; + FAILED = 2; + CANCELLED = 3; + } + + /** + * Phases of a backup session in RUNNING state + * + */ + enum BackupPhase { + REQUEST = 0; + SNAPSHOT = 1; + PREPARE_INCREMENTAL = 2; + SNAPSHOTCOPY = 3; + INCREMENTAL_COPY = 4; + STORE_MANIFEST = 5; + } +} \ No newline at end of file diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 09a4512ddfa..1eba784ff19 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -672,6 +672,11 @@ + + org.apache.hadoop + hadoop-distcp + ${hadoop-two.version} + org.apache.hadoop hadoop-common @@ -757,6 +762,11 @@ ${hadoop-three.version} + + org.apache.hadoop + hadoop-distcp + ${hadoop-three.version} + org.apache.hadoop hadoop-common diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java new file mode 100644 index 00000000000..6f642a443b8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * The administrative API for HBase Backup. Construct an instance and call {@link #close()} + * afterwards. + *

+ * BackupAdmin can be used to create backups, restore data from backups and for other + * backup-related operations. + * @since 2.0 + */ +@InterfaceAudience.Private +public interface BackupAdmin extends Closeable { + + /** + * Backup given list of tables fully. This is a synchronous operation. + * It returns backup id on success or throw exception on failure. + * @param userRequest BackupRequest instance + * @return the backup Id + */ + + String backupTables(final BackupRequest userRequest) throws IOException; + + /** + * Restore backup + * @param request restore request + * @throws IOException exception + */ + void restore(RestoreRequest request) throws IOException; + + /** + * Describe backup image command + * @param backupId backup id + * @return backup info + * @throws IOException exception + */ + BackupInfo getBackupInfo(String backupId) throws IOException; + + + /** + * Delete backup image command + * @param backupIds backup id list + * @return total number of deleted sessions + * @throws IOException exception + */ + int deleteBackups(String[] backupIds) throws IOException; + + /** + * Show backup history command + * @param n last n backup sessions + * @return list of backup info objects + * @throws IOException exception + */ + List getHistory(int n) throws IOException; + + /** + * Show backup history command with filters + * @param n last n backup sessions + * @param f list of filters + * @return list of backup info objects + * @throws IOException exception + */ + List getHistory(int n, BackupInfo.Filter... f) throws IOException; + + /** + * Backup sets list command - list all backup sets. Backup set is a named group of tables. + * @return all registered backup sets + * @throws IOException exception + */ + List listBackupSets() throws IOException; + + /** + * Backup set describe command. Shows list of tables in this particular backup set. + * @param name set name + * @return backup set description or null + * @throws IOException exception + */ + BackupSet getBackupSet(String name) throws IOException; + + /** + * Delete backup set command + * @param name backup set name + * @return true, if success, false - otherwise + * @throws IOException exception + */ + boolean deleteBackupSet(String name) throws IOException; + + /** + * Add tables to backup set command + * @param name name of backup set. + * @param tables list of tables to be added to this set. + * @throws IOException exception + */ + void addToBackupSet(String name, TableName[] tables) throws IOException; + + /** + * Remove tables from backup set + * @param name name of backup set. + * @param tables list of tables to be removed from this set. + * @throws IOException exception + */ + void removeFromBackupSet(String name, TableName[] tables) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java new file mode 100644 index 00000000000..007e4c1c858 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Backup copy job is a part of a backup process. The concrete implementation is responsible for + * copying data from a cluster to backup destination. Concrete implementation is provided by backup + * provider, see {@link BackupRestoreFactory} + */ +@InterfaceAudience.Private +public interface BackupCopyJob extends Configurable { + + /** + * Copy backup data to destination + * @param backupInfo context object + * @param backupManager backup manager + * @param conf configuration + * @param backupType backup type (FULL or INCREMENTAL) + * @param options array of options (implementation-specific) + * @return result (0 - success, -1 failure ) + * @throws IOException exception + */ + int copy(BackupInfo backupInfo, BackupManager backupManager, Configuration conf, + BackupType backupType, String[] options) throws IOException; + + /** + * Cancel copy job + * @param jobHandler backup copy job handler + * @throws IOException + */ + void cancel(String jobHandler) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java new file mode 100644 index 00000000000..5794fcea999 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; + +import java.io.IOException; +import java.net.URI; + +import org.apache.commons.cli.CommandLine; +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.backup.BackupRestoreConstants.BackupCommand; +import org.apache.hadoop.hbase.backup.impl.BackupCommands; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.util.ToolRunner; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +/** + * + * Command-line entry point for backup operation + * + */ +@InterfaceAudience.Private +public class BackupDriver extends AbstractHBaseTool { + + private static final Log LOG = LogFactory.getLog(BackupDriver.class); + private CommandLine cmd; + + public BackupDriver() throws IOException { + init(); + } + + protected void init() throws IOException { + // disable irrelevant loggers to avoid it mess up command output + LogUtils.disableZkAndClientLoggers(LOG); + } + + private int parseAndRun(String[] args) throws IOException { + + // Check if backup is enabled + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + + String cmd = null; + String[] remainArgs = null; + if (args == null || args.length == 0) { + printToolUsage(); + return -1; + } else { + cmd = args[0]; + remainArgs = new String[args.length - 1]; + if (args.length > 1) { + System.arraycopy(args, 1, remainArgs, 0, args.length - 1); + } + } + + BackupCommand type = BackupCommand.HELP; + if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.CREATE; + } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.HELP; + } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.DELETE; + } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.DESCRIBE; + } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.HISTORY; + } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.PROGRESS; + } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) { + type = BackupCommand.SET; + } else { + System.out.println("Unsupported command for backup: " + cmd); + printToolUsage(); + return -1; + } + + // enable debug logging + Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); + if (this.cmd.hasOption(OPTION_DEBUG)) { + backupClientLogger.setLevel(Level.DEBUG); + } else { + backupClientLogger.setLevel(Level.INFO); + } + + BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd); + if (type == BackupCommand.CREATE && conf != null) { + ((BackupCommands.CreateCommand) command).setConf(conf); + } + try { + command.execute(); + } catch (IOException e) { + if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) { + return -1; + } + throw e; + } + return 0; + } + + @Override + protected void addOptions() { + // define supported options + addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); + addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC); + addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC); + addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC); + addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC); + addOptWithArg(OPTION_SET, OPTION_SET_DESC); + addOptWithArg(OPTION_PATH, OPTION_PATH_DESC); + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + return parseAndRun(cmd.getArgs()); + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + Path hbasedir = FSUtils.getRootDir(conf); + URI defaultFs = hbasedir.getFileSystem(conf).getUri(); + FSUtils.setFsDefault(conf, new Path(defaultFs)); + int ret = ToolRunner.run(conf, new BackupDriver(), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws IOException { + if (conf == null) { + LOG.error("Tool configuration is not initialized"); + throw new NullPointerException("conf"); + } + + CommandLine cmd; + try { + // parse the command line arguments + cmd = parseArgs(args); + cmdLineArgs = args; + } catch (Exception e) { + System.err.println("Error when parsing command-line arguments: " + e.getMessage()); + printToolUsage(); + return EXIT_FAILURE; + } + processOptions(cmd); + + int ret = EXIT_FAILURE; + try { + ret = doWork(); + } catch (Exception e) { + LOG.error("Error running command-line tool", e); + return EXIT_FAILURE; + } + return ret; + } + + protected void printToolUsage() throws IOException { + System.out.println(BackupCommands.USAGE); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java new file mode 100644 index 00000000000..c7d2848ab56 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -0,0 +1,545 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder; +import org.apache.hadoop.hbase.util.Bytes; + + +/** + * An object to encapsulate the information for each backup session + */ +@InterfaceAudience.Private +public class BackupInfo implements Comparable { + private static final Log LOG = LogFactory.getLog(BackupInfo.class); + + public static interface Filter { + + /** + * Filter interface + * @param info backup info + * @return true if info passes filter, false otherwise + */ + public boolean apply(BackupInfo info); + } + + /** + * Backup session states + */ + public static enum BackupState { + RUNNING, COMPLETE, FAILED, ANY; + } + + /** + * BackupPhase - phases of an ACTIVE backup session (running), when state of a backup session is + * BackupState.RUNNING + */ + public static enum BackupPhase { + REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST; + } + + /** + * Backup id + */ + private String backupId; + + /** + * Backup type, full or incremental + */ + private BackupType type; + + /** + * Target root directory for storing the backup files + */ + private String backupRootDir; + + /** + * Backup state + */ + private BackupState state; + + /** + * Backup phase + */ + private BackupPhase phase; + + /** + * Backup failure message + */ + private String failedMsg; + + /** + * Backup status map for all tables + */ + private Map backupTableInfoMap; + + /** + * Actual start timestamp of a backup process + */ + private long startTs; + + /** + * Actual end timestamp of the backup process + */ + private long completeTs; + + /** + * Total bytes of incremental logs copied + */ + private long totalBytesCopied; + + /** + * For incremental backup, a location of a backed-up hlogs + */ + private String hlogTargetDir = null; + + /** + * Incremental backup file list + */ + private List incrBackupFileList; + + /** + * New region server log timestamps for table set after distributed log roll key - table name, + * value - map of RegionServer hostname -> last log rolled timestamp + */ + private HashMap> tableSetTimestampMap; + + /** + * Backup progress in %% (0-100) + */ + private int progress; + + /** + * Number of parallel workers. -1 - system defined + */ + private int workers = -1; + + /** + * Bandwidth per worker in MB per sec. -1 - unlimited + */ + private long bandwidth = -1; + + public BackupInfo() { + backupTableInfoMap = new HashMap(); + } + + public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) { + this(); + this.backupId = backupId; + this.type = type; + this.backupRootDir = targetRootDir; + this.addTables(tables); + if (type == BackupType.INCREMENTAL) { + setHLogTargetDir(BackupUtils.getLogBackupDir(targetRootDir, backupId)); + } + this.startTs = 0; + this.completeTs = 0; + } + + public int getWorkers() { + return workers; + } + + public void setWorkers(int workers) { + this.workers = workers; + } + + public long getBandwidth() { + return bandwidth; + } + + public void setBandwidth(long bandwidth) { + this.bandwidth = bandwidth; + } + + public void setBackupTableInfoMap(Map backupTableInfoMap) { + this.backupTableInfoMap = backupTableInfoMap; + } + + public HashMap> getTableSetTimestampMap() { + return tableSetTimestampMap; + } + + public void setTableSetTimestampMap(HashMap> tableSetTimestampMap) { + this.tableSetTimestampMap = tableSetTimestampMap; + } + + public void setType(BackupType type) { + this.type = type; + } + + public void setBackupRootDir(String targetRootDir) { + this.backupRootDir = targetRootDir; + } + + public void setTotalBytesCopied(long totalBytesCopied) { + this.totalBytesCopied = totalBytesCopied; + } + + /** + * Set progress (0-100%) + * @param p progress value + */ + + public void setProgress(int p) { + this.progress = p; + } + + /** + * Get current progress + */ + public int getProgress() { + return progress; + } + + public String getBackupId() { + return backupId; + } + + public void setBackupId(String backupId) { + this.backupId = backupId; + } + + public BackupTableInfo getBackupTableInfo(TableName table) { + return this.backupTableInfoMap.get(table); + } + + public String getFailedMsg() { + return failedMsg; + } + + public void setFailedMsg(String failedMsg) { + this.failedMsg = failedMsg; + } + + public long getStartTs() { + return startTs; + } + + public void setStartTs(long startTs) { + this.startTs = startTs; + } + + public long getCompleteTs() { + return completeTs; + } + + public void setCompleteTs(long endTs) { + this.completeTs = endTs; + } + + public long getTotalBytesCopied() { + return totalBytesCopied; + } + + public BackupState getState() { + return state; + } + + public void setState(BackupState flag) { + this.state = flag; + } + + public BackupPhase getPhase() { + return phase; + } + + public void setPhase(BackupPhase phase) { + this.phase = phase; + } + + public BackupType getType() { + return type; + } + + public void setSnapshotName(TableName table, String snapshotName) { + this.backupTableInfoMap.get(table).setSnapshotName(snapshotName); + } + + public String getSnapshotName(TableName table) { + return this.backupTableInfoMap.get(table).getSnapshotName(); + } + + public List getSnapshotNames() { + List snapshotNames = new ArrayList(); + for (BackupTableInfo backupStatus : this.backupTableInfoMap.values()) { + snapshotNames.add(backupStatus.getSnapshotName()); + } + return snapshotNames; + } + + public Set getTables() { + return this.backupTableInfoMap.keySet(); + } + + public List getTableNames() { + return new ArrayList(backupTableInfoMap.keySet()); + } + + public void addTables(TableName[] tables) { + for (TableName table : tables) { + BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId); + this.backupTableInfoMap.put(table, backupStatus); + } + } + + public void setTables(List tables) { + this.backupTableInfoMap.clear(); + for (TableName table : tables) { + BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId); + this.backupTableInfoMap.put(table, backupStatus); + } + } + + public String getBackupRootDir() { + return backupRootDir; + } + + public String getTableBackupDir(TableName tableName) { + return BackupUtils.getTableBackupDir(backupRootDir, backupId, tableName); + } + + public void setHLogTargetDir(String hlogTagetDir) { + this.hlogTargetDir = hlogTagetDir; + } + + public String getHLogTargetDir() { + return hlogTargetDir; + } + + public List getIncrBackupFileList() { + return incrBackupFileList; + } + + public void setIncrBackupFileList(List incrBackupFileList) { + this.incrBackupFileList = incrBackupFileList; + } + + /** + * Set the new region server log timestamps after distributed log roll + * @param 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; + } + + public TableName getTableBySnapshot(String snapshotName) { + for (Entry entry : this.backupTableInfoMap.entrySet()) { + if (snapshotName.equals(entry.getValue().getSnapshotName())) { + return entry.getKey(); + } + } + return null; + } + + public BackupProtos.BackupInfo toProtosBackupInfo() { + BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder(); + builder.setBackupId(getBackupId()); + setBackupTableInfoMap(builder); + builder.setCompleteTs(getCompleteTs()); + if (getFailedMsg() != null) { + builder.setFailedMessage(getFailedMsg()); + } + if (getState() != null) { + builder.setBackupState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name())); + } + if (getPhase() != null) { + builder.setBackupPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name())); + } + + builder.setProgress(getProgress()); + builder.setStartTs(getStartTs()); + builder.setBackupRootDir(getBackupRootDir()); + builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name())); + builder.setWorkersNumber(workers); + builder.setBandwidth(bandwidth); + return builder.build(); + } + + @Override + public int hashCode() { + int hash = 33 * type.hashCode() + backupId != null ? backupId.hashCode() : 0; + if (backupRootDir != null) { + hash = 33 * hash + backupRootDir.hashCode(); + } + hash = 33 * hash + state.hashCode(); + hash = 33 * hash + phase.hashCode(); + hash = 33 * hash + (int) (startTs ^ (startTs >>> 32)); + hash = 33 * hash + (int) (completeTs ^ (completeTs >>> 32)); + hash = 33 * hash + (int) (totalBytesCopied ^ (totalBytesCopied >>> 32)); + if (hlogTargetDir != null) { + hash = 33 * hash + hlogTargetDir.hashCode(); + } + return hash; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BackupInfo) { + BackupInfo other = (BackupInfo) obj; + try { + return Bytes.equals(toByteArray(), other.toByteArray()); + } catch (IOException e) { + LOG.error(e); + return false; + } + } else { + return false; + } + } + + public byte[] toByteArray() throws IOException { + return toProtosBackupInfo().toByteArray(); + } + + private void setBackupTableInfoMap(Builder builder) { + for (Entry entry : backupTableInfoMap.entrySet()) { + builder.addBackupTableInfo(entry.getValue().toProto()); + } + } + + public static BackupInfo fromByteArray(byte[] data) throws IOException { + return fromProto(BackupProtos.BackupInfo.parseFrom(data)); + } + + public static BackupInfo fromStream(final InputStream stream) throws IOException { + return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream)); + } + + public static BackupInfo fromProto(BackupProtos.BackupInfo proto) { + BackupInfo context = new BackupInfo(); + context.setBackupId(proto.getBackupId()); + context.setBackupTableInfoMap(toMap(proto.getBackupTableInfoList())); + context.setCompleteTs(proto.getCompleteTs()); + if (proto.hasFailedMessage()) { + context.setFailedMsg(proto.getFailedMessage()); + } + if (proto.hasBackupState()) { + context.setState(BackupInfo.BackupState.valueOf(proto.getBackupState().name())); + } + + context.setHLogTargetDir(BackupUtils.getLogBackupDir(proto.getBackupRootDir(), + proto.getBackupId())); + + if (proto.hasBackupPhase()) { + context.setPhase(BackupPhase.valueOf(proto.getBackupPhase().name())); + } + if (proto.hasProgress()) { + context.setProgress(proto.getProgress()); + } + context.setStartTs(proto.getStartTs()); + context.setBackupRootDir(proto.getBackupRootDir()); + context.setType(BackupType.valueOf(proto.getBackupType().name())); + context.setWorkers(proto.getWorkersNumber()); + context.setBandwidth(proto.getBandwidth()); + return context; + } + + private static Map toMap(List list) { + HashMap map = new HashMap<>(); + for (BackupProtos.BackupTableInfo tbs : list) { + map.put(ProtobufUtil.toTableName(tbs.getTableName()), BackupTableInfo.convert(tbs)); + } + return map; + } + + public String getShortDescription() { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + sb.append("ID=" + backupId).append(","); + sb.append("Type=" + getType()).append(","); + sb.append("Tables=" + getTableListAsString()).append(","); + sb.append("State=" + getState()).append(","); + Date date = null; + Calendar cal = Calendar.getInstance(); + cal.setTimeInMillis(getStartTs()); + date = cal.getTime(); + sb.append("Start time=" + date).append(","); + if (state == BackupState.FAILED) { + sb.append("Failed message=" + getFailedMsg()).append(","); + } else if (state == BackupState.RUNNING) { + sb.append("Phase=" + getPhase()).append(","); + } else if (state == BackupState.COMPLETE) { + cal = Calendar.getInstance(); + cal.setTimeInMillis(getCompleteTs()); + date = cal.getTime(); + sb.append("End time=" + date).append(","); + } + sb.append("Progress=" + getProgress() + "%"); + sb.append("}"); + + return sb.toString(); + } + + public String getStatusAndProgressAsString() { + StringBuilder sb = new StringBuilder(); + sb.append("id: ").append(getBackupId()).append(" state: ").append(getState()) + .append(" progress: ").append(getProgress()); + return sb.toString(); + } + + public String getTableListAsString() { + StringBuffer sb = new StringBuffer(); + sb.append("{"); + sb.append(StringUtils.join(backupTableInfoMap.keySet(), ",")); + sb.append("}"); + return sb.toString(); + } + + /** + * We use only time stamps to compare objects during sort operation + */ + @Override + public int compareTo(BackupInfo o) { + Long thisTS = + Long.valueOf(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1)); + Long otherTS = Long.valueOf(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1)); + return thisTS.compareTo(otherTS); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java new file mode 100644 index 00000000000..dadd861f495 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java @@ -0,0 +1,139 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.util.List; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * POJO class for backup request + */ +@InterfaceAudience.Private +public final class BackupRequest { + + public static class Builder { + + BackupRequest request; + + public Builder() { + request = new BackupRequest(); + } + + public Builder withBackupType(BackupType type) { + request.setBackupType(type); + return this; + } + + public Builder withTableList(List tables) { + request.setTableList(tables); + return this; + } + + public Builder withTargetRootDir(String backupDir) { + request.setTargetRootDir(backupDir); + return this; + } + + public Builder withBackupSetName(String setName) { + request.setBackupSetName(setName); + return this; + } + + public Builder withTotalTasks(int numTasks) { + request.setTotalTasks(numTasks); + return this; + } + + public Builder withBandwidthPerTasks(int bandwidth) { + request.setBandwidth(bandwidth); + return this; + } + + public BackupRequest build() { + return request; + } + + } + + private BackupType type; + private List tableList; + private String targetRootDir; + private int totalTasks = -1; + private long bandwidth = -1L; + private String backupSetName; + + private BackupRequest() { + } + + private BackupRequest setBackupType(BackupType type) { + this.type = type; + return this; + } + + public BackupType getBackupType() { + return this.type; + } + + private BackupRequest setTableList(List tableList) { + this.tableList = tableList; + return this; + } + + public List getTableList() { + return this.tableList; + } + + private BackupRequest setTargetRootDir(String targetRootDir) { + this.targetRootDir = targetRootDir; + return this; + } + + public String getTargetRootDir() { + return this.targetRootDir; + } + + private BackupRequest setTotalTasks(int totalTasks) { + this.totalTasks = totalTasks; + return this; + } + + public int getTotalTasks() { + return this.totalTasks; + } + + private BackupRequest setBandwidth(long bandwidth) { + this.bandwidth = bandwidth; + return this; + } + + public long getBandwidth() { + return this.bandwidth; + } + + public String getBackupSetName() { + return backupSetName; + } + + private BackupRequest setBackupSetName(String backupSetName) { + this.backupSetName = backupSetName; + return this; + } +} 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..770ccce75bc --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants + */ +@InterfaceAudience.Private +public interface BackupRestoreConstants { + /* + * Backup/Restore constants + */ + public final static String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name"; + public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "hbase:backup"; + + public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl"; + + public final static int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER; + public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable"; + public final static boolean BACKUP_ENABLE_DEFAULT = false; + /* + * Drivers option list + */ + public static final String OPTION_OVERWRITE = "o"; + public static final String OPTION_OVERWRITE_DESC = + "Overwrite data if any of the restore target tables exists"; + + public static final String OPTION_CHECK = "c"; + public static final String OPTION_CHECK_DESC = + "Check restore sequence and dependencies only (does not execute the command)"; + + public static final String OPTION_SET = "s"; + public static final String OPTION_SET_DESC = "Backup set name"; + public static final String OPTION_SET_RESTORE_DESC = + "Backup set to restore, mutually exclusive with -t (table list)"; + public static final String OPTION_SET_BACKUP_DESC = + "Backup set to backup, mutually exclusive with -t (table list)"; + public static final String OPTION_DEBUG = "d"; + public static final String OPTION_DEBUG_DESC = "Enable debug loggings"; + + public static final String OPTION_TABLE = "t"; + public static final String OPTION_TABLE_DESC = "Table name. If specified, only backup images," + + " which contain this table will be listed."; + + public static final String OPTION_TABLE_LIST = "l"; + public static final String OPTION_TABLE_LIST_DESC = "Table name list, comma-separated."; + + public static final String OPTION_BANDWIDTH = "b"; + public static final String OPTION_BANDWIDTH_DESC = "Bandwidth per task (MapReduce task) in MB/s"; + + public static final String OPTION_WORKERS = "w"; + public static final String OPTION_WORKERS_DESC = "Number of parallel MapReduce tasks to execute"; + + public static final String OPTION_RECORD_NUMBER = "n"; + public static final String OPTION_RECORD_NUMBER_DESC = + "Number of records of backup history. Default: 10"; + + public static final String OPTION_PATH = "p"; + public static final String OPTION_PATH_DESC = "Backup destination root directory path"; + + public static final String OPTION_TABLE_MAPPING = "m"; + public static final String OPTION_TABLE_MAPPING_DESC = + "A comma separated list of target tables. " + + "If specified, each table in must have a mapping"; + + public static final String BACKUP_CONFIG_STRING = BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n" + + "hbase.master.logcleaner.plugins=" + +"YOUR_PLUGINS,org.apache.hadoop.hbase.backup.master.BackupLogCleaner\n" + + "hbase.procedure.master.classes=YOUR_CLASSES," + +"org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager\n" + + "hbase.procedure.regionserver.classes=YOUR_CLASSES," + + "org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager\n" + + "and restart the cluster"; + public static final String ENABLE_BACKUP = "Backup is not enabled. To enable backup, "+ + "in hbase-site.xml, set:\n " + + BACKUP_CONFIG_STRING; + + public static final String VERIFY_BACKUP = "Please make sure that backup is enabled on the cluster. To enable backup, "+ + "in hbase-site.xml, set:\n " + + BACKUP_CONFIG_STRING; + + /* + * Delimiter in table name list in restore command + */ + public static final String TABLENAME_DELIMITER_IN_COMMAND = ","; + + public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root"; + + public static final String BACKUPID_PREFIX = "backup_"; + + public static enum BackupCommand { + CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS, + SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java new file mode 100644 index 00000000000..6d8967a06e6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.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.conf.Configuration; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob; +import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * Factory implementation for backup/restore related jobs + * + */ +@InterfaceAudience.Private +public final class BackupRestoreFactory { + + public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class"; + public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class"; + + private BackupRestoreFactory() { + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Gets backup restore job + * @param conf configuration + * @return backup restore task instance + */ + public static RestoreJob getRestoreJob(Configuration conf) { + Class cls = + conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreJob.class, RestoreJob.class); + RestoreJob service = ReflectionUtils.newInstance(cls, conf); + service.setConf(conf); + return service; + } + + /** + * Gets backup copy job + * @param conf configuration + * @return backup copy task + */ + public static BackupCopyJob getBackupCopyJob(Configuration conf) { + Class cls = + conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyJob.class, + BackupCopyJob.class); + BackupCopyJob service = ReflectionUtils.newInstance(cls, conf); + service.setConf(conf); + return service; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java new file mode 100644 index 00000000000..a5390a685b5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; + +/** + * Backup related information encapsulated for a table. At this moment only target directory, + * snapshot name and table name are encapsulated here. + */ + +@InterfaceAudience.Private +public class BackupTableInfo { + + /* + * Table name for backup + */ + private TableName table; + + /* + * Snapshot name for offline/online snapshot + */ + + private String snapshotName = null; + + public BackupTableInfo() { + + } + + public BackupTableInfo(TableName table, String targetRootDir, String backupId) { + this.table = table; + } + + public String getSnapshotName() { + return snapshotName; + } + + public void setSnapshotName(String snapshotName) { + this.snapshotName = snapshotName; + } + + public TableName getTable() { + return table; + } + + public static BackupTableInfo convert(BackupProtos.BackupTableInfo proto) { + BackupTableInfo bs = new BackupTableInfo(); + bs.table = ProtobufUtil.toTableName(proto.getTableName()); + if (proto.hasSnapshotName()) { + bs.snapshotName = proto.getSnapshotName(); + } + return bs; + } + + public BackupProtos.BackupTableInfo toProto() { + BackupProtos.BackupTableInfo.Builder builder = BackupProtos.BackupTableInfo.newBuilder(); + if (snapshotName != null) { + builder.setSnapshotName(snapshotName); + } + builder.setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toProtoTableName(table)); + return builder.build(); + } +} 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..46044db61eb --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -0,0 +1,141 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; +import java.util.HashMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * View to an on-disk Backup Image FileSytem Provides the set of methods necessary to interact with + * the on-disk Backup Image data. + */ +@InterfaceAudience.Private +public class HBackupFileSystem { + public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class); + + /** + * This is utility class. + */ + private HBackupFileSystem() { + } + + /** + * Given the backup root dir, backup id and the table name, return the backup image location, + * which is also where the backup manifest file is. return value look like: + * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/", + * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory + * @param backupRootDir backup root directory + * @param backupId backup id + * @param tableName table name + * @return backupPath String for the particular table + */ + public static String + getTableBackupDir(String backupRootDir, String backupId, TableName tableName) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString() + + Path.SEPARATOR; + } + + /** + * Given the backup root dir, backup id and the table name, return the backup image location, + * which is also where the backup manifest file is. return value look like: + * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/", + * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory + * @param backupRootPath backup root path + * @param tableName table name + * @param backupId backup Id + * @return backupPath for the particular table + */ + public static Path getTableBackupPath(TableName tableName, + Path backupRootPath, String backupId) { + return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName)); + } + + /** + * Given the backup root dir and the backup id, return the log file location for an incremental + * backup. + * @param backupRootDir backup root directory + * @param backupId backup id + * @return logBackupDir: ".../user/biadmin/backup/WALs/backup_1396650096738" + */ + public static String getLogBackupDir(String backupRootDir, String backupId) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + HConstants.HREGION_LOGDIR_NAME; + } + + public static Path getLogBackupPath(String backupRootDir, String backupId) { + return new Path(getLogBackupDir(backupRootDir, backupId)); + } + + private static Path getManifestPath(TableName tableName, Configuration conf, Path backupRootPath, + String backupId) throws IOException { + Path manifestPath = + new Path(getTableBackupPath(tableName, backupRootPath, backupId), + BackupManifest.MANIFEST_FILE_NAME); + + FileSystem fs = backupRootPath.getFileSystem(conf); + if (!fs.exists(manifestPath)) { + // check log dir for incremental backup case + manifestPath = + new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR + + BackupManifest.MANIFEST_FILE_NAME); + if (!fs.exists(manifestPath)) { + String errorMsg = + "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for " + + backupId + ". File " + manifestPath + " does not exists. Did " + backupId + + " correspond to previously taken backup ?"; + throw new IOException(errorMsg); + } + } + return manifestPath; + } + + public static BackupManifest getManifest(TableName tableName, Configuration conf, + Path backupRootPath, String backupId) throws IOException { + BackupManifest manifest = + new BackupManifest(conf, getManifestPath(tableName, conf, backupRootPath, backupId)); + return manifest; + } + + /** + * Check whether the backup image path and there is manifest file in the path. + * @param backupManifestMap If all the manifests are found, then they are put into this map + * @param tableArray the tables involved + * @throws IOException exception + */ + public static void checkImageManifestExist(HashMap backupManifestMap, + TableName[] tableArray, Configuration conf, Path backupRootPath, String backupId) + throws IOException { + for (TableName tableName : tableArray) { + BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId); + backupManifestMap.put(tableName, manifest); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java new file mode 100644 index 00000000000..1becb75c5d2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import org.apache.commons.logging.Log; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +/** + * Utility class for disabling Zk and client logging + * + */ +@InterfaceAudience.Private +final class LogUtils { + + private LogUtils() { + } + + /** + * Disables Zk- and HBase client logging + * @param log + */ + static void disableZkAndClientLoggers(Log log) { + // disable zookeeper log to avoid it mess up command output + Logger zkLogger = Logger.getLogger("org.apache.zookeeper"); + zkLogger.setLevel(Level.OFF); + // disable hbase zookeeper tool log to avoid it mess up command output + Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper"); + hbaseZkLogger.setLevel(Level.OFF); + // disable hbase client log to avoid it mess up command output + Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client"); + hbaseClientLogger.setLevel(Level.OFF); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java new file mode 100644 index 00000000000..82a1b563726 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -0,0 +1,265 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC; + +import java.io.IOException; +import java.net.URI; +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.util.ToolRunner; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +/** + * + * Command-line entry point for restore operation + * + */ +@InterfaceAudience.Private +public class RestoreDriver extends AbstractHBaseTool { + + private static final Log LOG = LogFactory.getLog(RestoreDriver.class); + private CommandLine cmd; + + private static final String USAGE_STRING = + "Usage: hbase restore [options]\n" + + " backup_path Path to a backup destination root\n" + + " backup_id Backup image ID to restore\n" + + " table(s) Comma-separated list of tables to restore\n"; + + private static final String USAGE_FOOTER = ""; + + protected RestoreDriver() throws IOException { + init(); + } + + protected void init() throws IOException { + // disable irrelevant loggers to avoid it mess up command output + LogUtils.disableZkAndClientLoggers(LOG); + } + + private int parseAndRun(String[] args) throws IOException { + // Check if backup is enabled + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + + // enable debug logging + Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup"); + if (cmd.hasOption(OPTION_DEBUG)) { + backupClientLogger.setLevel(Level.DEBUG); + } + + // whether to overwrite to existing table if any, false by default + boolean overwrite = cmd.hasOption(OPTION_OVERWRITE); + if (overwrite) { + LOG.debug("Found -overwrite option in restore command, " + + "will overwrite to existing table if any in the restore target"); + } + + // whether to only check the dependencies, false by default + boolean check = cmd.hasOption(OPTION_CHECK); + if (check) { + LOG.debug("Found -check option in restore command, " + + "will check and verify the dependencies"); + } + + if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) { + System.err.println("Options -s and -t are mutaully exclusive,"+ + " you can not specify both of them."); + printToolUsage(); + return -1; + } + + if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) { + System.err.println("You have to specify either set name or table list to restore"); + printToolUsage(); + return -1; + } + // parse main restore command options + String[] remainArgs = cmd.getArgs(); + if (remainArgs.length != 2) { + printToolUsage(); + return -1; + } + + String backupRootDir = remainArgs[0]; + String backupId = remainArgs[1]; + String tables = null; + String tableMapping = + cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null; + try (final Connection conn = ConnectionFactory.createConnection(conf); + BackupAdmin client = new BackupAdminImpl(conn);) { + // Check backup set + if (cmd.hasOption(OPTION_SET)) { + String setName = cmd.getOptionValue(OPTION_SET); + try { + tables = getTablesForSet(conn, setName, conf); + } catch (IOException e) { + System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName); + printToolUsage(); + return -2; + } + if (tables == null) { + System.out.println("ERROR: Backup set '" + setName + + "' is either empty or does not exist"); + printToolUsage(); + return -3; + } + } else { + tables = cmd.getOptionValue(OPTION_TABLE); + } + + TableName[] sTableArray = BackupUtils.parseTableNames(tables); + TableName[] tTableArray = BackupUtils.parseTableNames(tableMapping); + + if (sTableArray != null && tTableArray != null && + (sTableArray.length != tTableArray.length)) { + System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping); + printToolUsage(); + return -4; + } + + client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check, + sTableArray, tTableArray, overwrite)); + } catch (Exception e) { + e.printStackTrace(); + return -5; + } + return 0; + } + + private String getTablesForSet(Connection conn, String name, Configuration conf) + throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List tables = table.describeBackupSet(name); + if (tables == null) return null; + return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } + + @Override + protected void addOptions() { + // define supported options + addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC); + addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC); + addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); + addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC); + addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC); + + addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC); + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + return parseAndRun(cmd.getArgs()); + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + Path hbasedir = FSUtils.getRootDir(conf); + URI defaultFs = hbasedir.getFileSystem(conf).getUri(); + FSUtils.setFsDefault(conf, new Path(defaultFs)); + int ret = ToolRunner.run(conf, new RestoreDriver(), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws IOException { + if (conf == null) { + LOG.error("Tool configuration is not initialized"); + throw new NullPointerException("conf"); + } + + CommandLine cmd; + try { + // parse the command line arguments + cmd = parseArgs(args); + cmdLineArgs = args; + } catch (Exception e) { + System.out.println("Error when parsing command-line arguments: " + e.getMessage()); + printToolUsage(); + return EXIT_FAILURE; + } + + if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) { + printToolUsage(); + return EXIT_FAILURE; + } + + processOptions(cmd); + + int ret = EXIT_FAILURE; + try { + ret = doWork(); + } catch (Exception e) { + LOG.error("Error running command-line tool", e); + return EXIT_FAILURE; + } + return ret; + } + + protected void printToolUsage() throws IOException { + System.out.println(USAGE_STRING); + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java new file mode 100644 index 00000000000..86fb963ddbf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Restore operation job interface Concrete implementation is provided by backup provider, see + * {@link BackupRestoreFactory} + */ + +@InterfaceAudience.Private +public interface RestoreJob extends Configurable { + + /** + * Run restore operation + * @param dirPaths path array of WAL log directories + * @param fromTables from tables + * @param toTables to tables + * @param fullBackupRestore full backup restore + * @throws IOException + */ + void run(Path[] dirPaths, TableName[] fromTables, TableName[] toTables, + boolean fullBackupRestore) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java new file mode 100644 index 00000000000..de3ad5ae3c9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * POJO class for restore request + */ +@InterfaceAudience.Private +public class RestoreRequest { + + public static class Builder { + RestoreRequest request; + + public Builder() { + request = new RestoreRequest(); + } + + public Builder withBackupRootDir(String backupRootDir) { + request.setBackupRootDir(backupRootDir); + return this; + } + + public Builder withBackupId(String backupId) { + request.setBackupId(backupId); + return this; + } + + public Builder withCheck(boolean check) { + request.setCheck(check); + return this; + } + + public Builder withFromTables(TableName[] fromTables) { + request.setFromTables(fromTables); + return this; + } + + public Builder withToTables(TableName[] toTables) { + request.setToTables(toTables); + return this; + } + + public Builder withOvewrite(boolean overwrite) { + request.setOverwrite(overwrite); + return this; + } + + + public RestoreRequest build() { + return request; + } + } + + private String backupRootDir; + private String backupId; + private boolean check = false; + private TableName[] fromTables; + private TableName[] toTables; + private boolean overwrite = false; + + private RestoreRequest() { + } + + public String getBackupRootDir() { + return backupRootDir; + } + + private RestoreRequest setBackupRootDir(String backupRootDir) { + this.backupRootDir = backupRootDir; + return this; + } + + public String getBackupId() { + return backupId; + } + + private RestoreRequest setBackupId(String backupId) { + this.backupId = backupId; + return this; + } + + public boolean isCheck() { + return check; + } + + private RestoreRequest setCheck(boolean check) { + this.check = check; + return this; + } + + public TableName[] getFromTables() { + return fromTables; + } + + private RestoreRequest setFromTables(TableName[] fromTables) { + this.fromTables = fromTables; + return this; + } + + public TableName[] getToTables() { + return toTables; + } + + private RestoreRequest setToTables(TableName[] toTables) { + this.toTables = toTables; + return this; + } + + public boolean isOverwrite() { + return overwrite; + } + + private RestoreRequest setOverwrite(boolean overwrite) { + this.overwrite = overwrite; + return this; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java new file mode 100644 index 00000000000..c1d5258c6f2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -0,0 +1,524 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupAdmin; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +import com.google.common.collect.Lists; + +@InterfaceAudience.Private +public class BackupAdminImpl implements BackupAdmin { + public final static String CHECK_OK = "Checking backup images: OK"; + public final static String CHECK_FAILED = + "Checking backup images: Failed. Some dependencies are missing for restore"; + private static final Log LOG = LogFactory.getLog(BackupAdminImpl.class); + + private final Connection conn; + + public BackupAdminImpl(Connection conn) { + this.conn = conn; + } + + @Override + public void close() throws IOException { + if (conn != null) { + conn.close(); + } + } + + + @Override + public BackupInfo getBackupInfo(String backupId) throws IOException { + BackupInfo backupInfo = null; + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (backupId == null) { + ArrayList recentSessions = table.getBackupInfos(BackupState.RUNNING); + if (recentSessions.isEmpty()) { + LOG.warn("No ongoing sessions found."); + return null; + } + // else show status for ongoing session + // must be one maximum + return recentSessions.get(0); + } else { + backupInfo = table.readBackupInfo(backupId); + return backupInfo; + } + } + } + + @Override + public int deleteBackups(String[] backupIds) throws IOException { + // TODO: requires Fault tolerance support, failure will leave system + // in a non-consistent state + // see HBASE-15227 + int totalDeleted = 0; + Map> allTablesMap = new HashMap>(); + + try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) { + for (int i = 0; i < backupIds.length; i++) { + BackupInfo info = sysTable.readBackupInfo(backupIds[i]); + if (info != null) { + String rootDir = info.getBackupRootDir(); + HashSet allTables = allTablesMap.get(rootDir); + if (allTables == null) { + allTables = new HashSet(); + allTablesMap.put(rootDir, allTables); + } + allTables.addAll(info.getTableNames()); + totalDeleted += deleteBackup(backupIds[i], sysTable); + } + } + finalizeDelete(allTablesMap, sysTable); + } + return totalDeleted; + } + + /** + * Updates incremental backup set for every backupRoot + * @param tablesMap map [backupRoot: Set] + * @param table backup system table + * @throws IOException + */ + + private void finalizeDelete(Map> tablesMap, BackupSystemTable table) + throws IOException { + for (String backupRoot : tablesMap.keySet()) { + Set incrTableSet = table.getIncrementalBackupTableSet(backupRoot); + Map> tableMap = + table.getBackupHistoryForTableSet(incrTableSet, backupRoot); + for (Map.Entry> entry : tableMap.entrySet()) { + if (entry.getValue() == null) { + // No more backups for a table + incrTableSet.remove(entry.getKey()); + } + } + if (!incrTableSet.isEmpty()) { + table.addIncrementalBackupTableSet(incrTableSet, backupRoot); + } else { // empty + table.deleteIncrementalBackupTableSet(backupRoot); + } + } + } + + /** + * Delete single backup and all related backups
+ * Algorithm:
+ * Backup type: FULL or INCREMENTAL
+ * Is this last backup session for table T: YES or NO
+ * For every table T from table list 'tables':
+ * if(FULL, YES) deletes only physical data (PD)
+ * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,
+ * until we either reach the most recent backup for T in the system or FULL backup
+ * which includes T
+ * if(INCREMENTAL, YES) deletes only physical data (PD) + * if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images between last
+ * FULL backup, which is older than the backup being deleted and the next FULL backup (if exists)
+ * or last one for a particular table T and removes T from list of backup tables. + * @param backupId backup id + * @param sysTable backup system table + * @return total number of deleted backup images + * @throws IOException + */ + private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { + + BackupInfo backupInfo = sysTable.readBackupInfo(backupId); + + int totalDeleted = 0; + if (backupInfo != null) { + LOG.info("Deleting backup " + backupInfo.getBackupId() + " ..."); + BackupUtils.cleanupBackupData(backupInfo, conn.getConfiguration()); + // List of tables in this backup; + List tables = backupInfo.getTableNames(); + long startTime = backupInfo.getStartTs(); + for (TableName tn : tables) { + boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime); + if (isLastBackupSession) { + continue; + } + // else + List affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable); + for (BackupInfo info : affectedBackups) { + if (info.equals(backupInfo)) { + continue; + } + removeTableFromBackupImage(info, tn, sysTable); + } + } + LOG.debug("Delete backup info " + backupInfo.getBackupId()); + + sysTable.deleteBackupInfo(backupInfo.getBackupId()); + LOG.info("Delete backup " + backupInfo.getBackupId() + " completed."); + totalDeleted++; + } else { + LOG.warn("Delete backup failed: no information found for backupID=" + backupId); + } + return totalDeleted; + } + + private void removeTableFromBackupImage(BackupInfo info, TableName tn, + BackupSystemTable sysTable) throws IOException { + List tables = info.getTableNames(); + LOG.debug("Remove " + tn + " from " + info.getBackupId() + " tables=" + + info.getTableListAsString()); + if (tables.contains(tn)) { + tables.remove(tn); + + if (tables.isEmpty()) { + LOG.debug("Delete backup info " + info.getBackupId()); + + sysTable.deleteBackupInfo(info.getBackupId()); + BackupUtils.cleanupBackupData(info, conn.getConfiguration()); + } else { + info.setTables(tables); + sysTable.updateBackupInfo(info); + // Now, clean up directory for table + cleanupBackupDir(info, tn, conn.getConfiguration()); + } + } + } + + private List getAffectedBackupInfos(BackupInfo backupInfo, TableName tn, + BackupSystemTable table) throws IOException { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn); + long ts = backupInfo.getStartTs(); + List list = new ArrayList(); + List history = table.getBackupHistory(backupInfo.getBackupRootDir()); + // Scan from most recent to backupInfo + // break when backupInfo reached + for (BackupInfo info : history) { + if (info.getStartTs() == ts) { + break; + } + List tables = info.getTableNames(); + if (tables.contains(tn)) { + BackupType bt = info.getType(); + if (bt == BackupType.FULL) { + // Clear list if we encounter FULL backup + list.clear(); + } else { + LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn + + " added " + info.getBackupId() + " tables=" + info.getTableListAsString()); + list.add(info); + } + } + } + return list; + } + + /** + * Clean up the data at target directory + * @throws IOException + */ + private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) + throws IOException { + try { + // clean up the data at target directory + String targetDir = backupInfo.getBackupRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupInfo.getBackupId()); + return; + } + + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + Path targetDirPath = + new Path(BackupUtils.getTableBackupDir(backupInfo.getBackupRootDir(), + backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table + + "at " + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + throw e1; + } + } + + private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) + throws IOException { + List history = table.getBackupHistory(); + for (BackupInfo info : history) { + List tables = info.getTableNames(); + if (!tables.contains(tn)) { + continue; + } + if (info.getStartTs() <= startTime) { + return true; + } else { + return false; + } + } + return false; + } + + @Override + public List getHistory(int n) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + if (history.size() <= n) return history; + List list = new ArrayList(); + for (int i = 0; i < n; i++) { + list.add(history.get(i)); + } + return list; + } + } + + @Override + public List getHistory(int n, BackupInfo.Filter... filters) throws IOException { + if (filters.length == 0) return getHistory(n); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List history = table.getBackupHistory(); + List result = new ArrayList(); + for (BackupInfo bi : history) { + if (result.size() == n) break; + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(bi)) { + passed = false; + break; + } + } + if (passed) { + result.add(bi); + } + } + return result; + } + } + + @Override + public List listBackupSets() throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.listBackupSets(); + List bslist = new ArrayList(); + for (String s : list) { + List tables = table.describeBackupSet(s); + if (tables != null) { + bslist.add(new BackupSet(s, tables)); + } + } + return bslist; + } + } + + @Override + public BackupSet getBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List list = table.describeBackupSet(name); + if (list == null) return null; + return new BackupSet(name, list); + } + } + + @Override + public boolean deleteBackupSet(String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + if (table.describeBackupSet(name) == null) { + return false; + } + table.deleteBackupSet(name); + return true; + } + } + + @Override + public void addToBackupSet(String name, TableName[] tables) throws IOException { + String[] tableNames = new String[tables.length]; + try (final BackupSystemTable table = new BackupSystemTable(conn); + final Admin admin = conn.getAdmin();) { + for (int i = 0; i < tables.length; i++) { + tableNames[i] = tables[i].getNameAsString(); + if (!admin.tableExists(TableName.valueOf(tableNames[i]))) { + throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist"); + } + } + table.addToBackupSet(name, tableNames); + LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name + + "' backup set"); + } + } + + @Override + public void removeFromBackupSet(String name, TableName[] tables) throws IOException { + LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'"); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + table.removeFromBackupSet(name, toStringArray(tables)); + LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + + "' completed."); + } + } + + private String[] toStringArray(TableName[] list) { + String[] arr = new String[list.length]; + for(int i=0; i < list.length; i++) { + arr[i] = list[i].toString(); + } + return arr; + } + + @Override + public void restore(RestoreRequest request) throws IOException { + if (request.isCheck()) { + HashMap backupManifestMap = new HashMap<>(); + // check and load backup image manifest for the tables + Path rootPath = new Path(request.getBackupRootDir()); + String backupId = request.getBackupId(); + TableName[] sTableArray = request.getFromTables(); + HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, + conn.getConfiguration(), rootPath, backupId); + + // Check and validate the backup image and its dependencies + + if (BackupUtils.validate(backupManifestMap, conn.getConfiguration())) { + LOG.info(CHECK_OK); + } else { + LOG.error(CHECK_FAILED); + } + return; + } + // Execute restore request + new RestoreTablesClient(conn, request).execute(); + } + + @Override + public String backupTables(BackupRequest request) throws IOException { + BackupType type = request.getBackupType(); + String targetRootDir = request.getTargetRootDir(); + List tableList = request.getTableList(); + + String backupId =BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime(); + if (type == BackupType.INCREMENTAL) { + Set incrTableSet = null; + try (BackupSystemTable table = new BackupSystemTable(conn)) { + incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + } + + if (incrTableSet.isEmpty()) { + String msg = "Incremental backup table set contains no tables. " + + "You need to run full backup first " + + (tableList != null ? "on "+StringUtils.join(tableList, ","): ""); + + throw new IOException(msg); + } + if(tableList != null) { + tableList.removeAll(incrTableSet); + if (!tableList.isEmpty()) { + String extraTables = StringUtils.join(tableList, ","); + String msg = "Some tables (" + extraTables + ") haven't gone through full backup. "+ + "Perform full backup on " + extraTables + " first, " + + "then retry the command"; + throw new IOException(msg); + } + } + tableList = Lists.newArrayList(incrTableSet); + } + if (tableList != null && !tableList.isEmpty()) { + for (TableName table : tableList) { + String targetTableBackupDir = + HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table); + Path targetTableBackupDirPath = new Path(targetTableBackupDir); + FileSystem outputFs = + FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration()); + if (outputFs.exists(targetTableBackupDirPath)) { + throw new IOException("Target backup directory " + targetTableBackupDir + + " exists already."); + } + } + ArrayList nonExistingTableList = null; + try (Admin admin = conn.getAdmin();) { + for (TableName tableName : tableList) { + if (!admin.tableExists(tableName)) { + if (nonExistingTableList == null) { + nonExistingTableList = new ArrayList<>(); + } + nonExistingTableList.add(tableName); + } + } + } + if (nonExistingTableList != null) { + if (type == BackupType.INCREMENTAL) { + // Update incremental backup set + tableList = excludeNonExistingTables(tableList, nonExistingTableList); + } else { + // Throw exception only in full mode - we try to backup non-existing table + throw new IOException("Non-existing tables found in the table list: " + + nonExistingTableList); + } + } + } + + // update table list + BackupRequest.Builder builder = new BackupRequest.Builder(); + request = builder.withBackupType(request.getBackupType()). + withTableList(tableList). + withTargetRootDir(request.getTargetRootDir()). + withBackupSetName(request.getBackupSetName()). + withTotalTasks(request.getTotalTasks()). + withBandwidthPerTasks((int)request.getBandwidth()).build(); + + if (type == BackupType.FULL) { + new FullTableBackupClient(conn, backupId, request).execute(); + } else { + new IncrementalTableBackupClient(conn, backupId, request).execute(); + } + return backupId; + } + + private List excludeNonExistingTables(List tableList, + List nonExistingTableList) { + + for (TableName table : nonExistingTableList) { + tableList.remove(table); + } + return tableList; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java new file mode 100644 index 00000000000..75e0ab7a27e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -0,0 +1,780 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_BACKUP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; + +import java.io.IOException; +import java.net.URI; +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.backup.util.BackupSet; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; + +import com.google.common.collect.Lists; + +/** + * General backup commands, options and usage messages + */ + +@InterfaceAudience.Private +public final class BackupCommands { + + public final static String INCORRECT_USAGE = "Incorrect usage"; + + public static final String USAGE = "Usage: hbase backup COMMAND [command-specific arguments]\n" + + "where COMMAND is one of:\n" + " create create a new backup image\n" + + " delete delete an existing backup image\n" + + " describe show the detailed information of a backup image\n" + + " history show history of all successful backups\n" + + " progress show the progress of the latest backup request\n" + + " set backup set management\n" + + "Run \'hbase backup COMMAND -h\' to see help message for each command\n"; + + public static final String CREATE_CMD_USAGE = + "Usage: hbase backup create [options]\n" + + " type \"full\" to create a full backup image\n" + + " \"incremental\" to create an incremental backup image\n" + + " backup_path Full path to store the backup image\n"; + + public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress \n" + + " backup_id Backup image id (optional). If no id specified, the command will show\n" + + " progress for currently running backup session."; + public static final String NO_INFO_FOUND = "No info was found for backup id: "; + public static final String NO_ACTIVE_SESSION_FOUND = "No active backup sessions found."; + + public static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup describe \n" + + " backup_id Backup image id\n"; + + public static final String HISTORY_CMD_USAGE = "Usage: hbase backup history [options]"; + + public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete \n" + + " backup_id Backup image id\n"; + + public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel \n" + + " backup_id Backup image id\n"; + + public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n" + + " name Backup set name\n" + + " tables Comma separated list of tables.\n" + + "COMMAND is one of:\n" + " add add tables to a set, create a set if needed\n" + + " remove remove tables from a set\n" + + " list list all backup sets in the system\n" + + " describe describe set\n" + " delete delete backup set\n"; + + public static final String USAGE_FOOTER = ""; + + public static abstract class Command extends Configured { + CommandLine cmdline; + + Command(Configuration conf) { + super(conf); + } + + public void execute() throws IOException { + if (cmdline.hasOption("h") || cmdline.hasOption("help")) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + protected abstract void printUsage(); + } + + private BackupCommands() { + throw new AssertionError("Instantiating utility class..."); + } + + public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) { + Command cmd = null; + switch (type) { + case CREATE: + cmd = new CreateCommand(conf, cmdline); + break; + case DESCRIBE: + cmd = new DescribeCommand(conf, cmdline); + break; + case PROGRESS: + cmd = new ProgressCommand(conf, cmdline); + break; + case DELETE: + cmd = new DeleteCommand(conf, cmdline); + break; + case CANCEL: + cmd = new CancelCommand(conf, cmdline); + break; + case HISTORY: + cmd = new HistoryCommand(conf, cmdline); + break; + case SET: + cmd = new BackupSetCommand(conf, cmdline); + break; + case HELP: + default: + cmd = new HelpCommand(conf, cmdline); + break; + } + return cmd; + } + + static int numOfArgs(String[] args) { + if (args == null) return 0; + return args.length; + } + + public static class CreateCommand extends Command { + + CreateCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + if (cmdline == null || cmdline.getArgs() == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String[] args = cmdline.getArgs(); + if (args.length !=3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + if (!BackupType.FULL.toString().equalsIgnoreCase(args[1]) + && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) { + System.out.println("ERROR: invalid backup type: " + args[1]); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + if (!verifyPath(args[2])) { + System.out.println("ERROR: invalid backup destination: " + args[2]); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String tables = null; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + + // Check if we have both: backup set and list of tables + if (cmdline.hasOption(OPTION_TABLE) && cmdline.hasOption(OPTION_SET)) { + System.out.println("ERROR: You can specify either backup set or list" + + " of tables, but not both"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + // Check backup set + String setName = null; + if (cmdline.hasOption(OPTION_SET)) { + setName = cmdline.getOptionValue(OPTION_SET); + tables = getTablesForSet(setName, conf); + + if (tables == null) { + System.out.println("ERROR: Backup set '" + setName + + "' is either empty or does not exist"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } else { + tables = cmdline.getOptionValue(OPTION_TABLE); + } + int bandwidth = + cmdline.hasOption(OPTION_BANDWIDTH) ? Integer.parseInt(cmdline + .getOptionValue(OPTION_BANDWIDTH)) : -1; + int workers = + cmdline.hasOption(OPTION_WORKERS) ? Integer.parseInt(cmdline + .getOptionValue(OPTION_WORKERS)) : -1; + + try (Connection conn = ConnectionFactory.createConnection(getConf()); + BackupAdminImpl admin = new BackupAdminImpl(conn);) { + + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase())) + .withTableList(tables != null ? + Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null) + .withTargetRootDir(args[2]) + .withTotalTasks(workers) + .withBandwidthPerTasks(bandwidth) + .withBackupSetName(setName).build(); + String backupId = admin.backupTables(request); + System.out.println("Backup session " + backupId + " finished. Status: SUCCESS"); + } catch (IOException e) { + System.out.println("Backup session finished. Status: FAILURE"); + throw e; + } + } + + private boolean verifyPath(String path) { + try { + Path p = new Path(path); + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + URI uri = p.toUri(); + if (uri.getScheme() == null) return false; + FileSystem.get(uri, conf); + return true; + } catch (Exception e) { + return false; + } + } + + private String getTablesForSet(String name, Configuration conf) throws IOException { + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable table = new BackupSystemTable(conn)) { + List tables = table.describeBackupSet(name); + if (tables == null) return null; + return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } + + @Override + protected void printUsage() { + System.out.println(CREATE_CMD_USAGE); + Options options = new Options(); + options.addOption(OPTION_WORKERS, true, OPTION_WORKERS_DESC); + options.addOption(OPTION_BANDWIDTH, true, OPTION_BANDWIDTH_DESC); + options.addOption(OPTION_SET, true, OPTION_SET_BACKUP_DESC); + options.addOption(OPTION_TABLE, true, OPTION_TABLE_LIST_DESC); + + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + + } + } + + private static class HelpCommand extends Command { + + HelpCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + if (cmdline == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] args = cmdline.getArgs(); + if (args == null || args.length == 0) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + if (args.length != 2) { + System.out.println("ERROR: Only supports help message of a single command type"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String type = args[1]; + + if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) { + System.out.println(CREATE_CMD_USAGE); + } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) { + System.out.println(DESCRIBE_CMD_USAGE); + } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) { + System.out.println(HISTORY_CMD_USAGE); + } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) { + System.out.println(PROGRESS_CMD_USAGE); + } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) { + System.out.println(DELETE_CMD_USAGE); + } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) { + System.out.println(CANCEL_CMD_USAGE); + } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) { + System.out.println(SET_CMD_USAGE); + } else { + System.out.println("Unknown command : " + type); + printUsage(); + } + } + + @Override + protected void printUsage() { + System.out.println(USAGE); + } + } + + private static class DescribeCommand extends Command { + + DescribeCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + if (cmdline == null || cmdline.getArgs() == null) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String[] args = cmdline.getArgs(); + if (args.length != 2) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String backupId = args[1]; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable sysTable = new BackupSystemTable(conn);) { + BackupInfo info = sysTable.readBackupInfo(backupId); + if (info == null) { + System.out.println("ERROR: " + backupId + " does not exist"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + System.out.println(info.getShortDescription()); + } + } + + @Override + protected void printUsage() { + System.out.println(DESCRIBE_CMD_USAGE); + } + } + + private static class ProgressCommand extends Command { + + ProgressCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 1) { + System.out.println("No backup id was specified, " + + "will retrieve the most recent (ongoing) session"); + } + String[] args = cmdline == null ? null : cmdline.getArgs(); + if (args != null && args.length > 2) { + System.err.println("ERROR: wrong number of arguments: " + args.length); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String backupId = (args == null || args.length <= 1) ? null : args[1]; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable sysTable = new BackupSystemTable(conn);) { + BackupInfo info = null; + + if (backupId != null) { + info = sysTable.readBackupInfo(backupId); + } else { + List infos = sysTable.getBackupInfos(BackupState.RUNNING); + if (infos != null && infos.size() > 0) { + info = infos.get(0); + backupId = info.getBackupId(); + System.out.println("Found ongoing session with backupId=" + backupId); + } else { + } + } + int progress = info == null ? -1 : info.getProgress(); + if (progress < 0) { + if (backupId != null) { + System.out.println(NO_INFO_FOUND + backupId); + } else { + System.err.println(NO_ACTIVE_SESSION_FOUND); + } + } else { + System.out.println(backupId + " progress=" + progress + "%"); + } + } + } + + @Override + protected void printUsage() { + System.out.println(PROGRESS_CMD_USAGE); + } + } + + private static class DeleteCommand extends Command { + + DeleteCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] args = cmdline.getArgs(); + String[] backupIds = new String[args.length - 1]; + System.arraycopy(args, 1, backupIds, 0, backupIds.length); + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + BackupAdminImpl admin = new BackupAdminImpl(conn);) { + int deleted = admin.deleteBackups(backupIds); + System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length); + } + + } + + @Override + protected void printUsage() { + System.out.println(DELETE_CMD_USAGE); + } + } + + // TODO Cancel command + + private static class CancelCommand extends Command { + + CancelCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + throw new UnsupportedOperationException("Cancel command is not supported yet."); + } + + @Override + protected void printUsage() { + } + } + + private static class HistoryCommand extends Command { + + private final static int DEFAULT_HISTORY_LENGTH = 10; + + HistoryCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + + super.execute(); + + int n = parseHistoryLength(); + final TableName tableName = getTableName(); + final String setName = getTableSetName(); + BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + if (tableName == null) return true; + List names = info.getTableNames(); + return names.contains(tableName); + } + }; + BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + if (setName == null) return true; + String backupId = info.getBackupId(); + return backupId.startsWith(setName); + } + }; + Path backupRootPath = getBackupRootPath(); + List history = null; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + if (backupRootPath == null) { + // Load from backup system table + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable sysTable = new BackupSystemTable(conn);) { + + history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter); + } + } else { + // load from backup FS + history = + BackupUtils.getHistory(conf, n, backupRootPath, tableNameFilter, tableSetFilter); + } + for (BackupInfo info : history) { + System.out.println(info.getShortDescription()); + } + } + + private Path getBackupRootPath() throws IOException { + String value = null; + try { + value = cmdline.getOptionValue(OPTION_PATH); + if (value == null) return null; + return new Path(value); + } catch (IllegalArgumentException e) { + System.out.println("ERROR: Illegal argument for backup root path: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + private TableName getTableName() throws IOException { + String value = cmdline.getOptionValue(OPTION_TABLE); + if (value == null) return null; + try { + return TableName.valueOf(value); + } catch (IllegalArgumentException e) { + System.out.println("Illegal argument for table name: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + private String getTableSetName() throws IOException { + String value = cmdline.getOptionValue(OPTION_SET); + return value; + } + + private int parseHistoryLength() throws IOException { + String value = cmdline.getOptionValue(OPTION_RECORD_NUMBER); + try { + if (value == null) return DEFAULT_HISTORY_LENGTH; + return Integer.parseInt(value); + } catch (NumberFormatException e) { + System.out.println("Illegal argument for history length: " + value); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + @Override + protected void printUsage() { + System.out.println(HISTORY_CMD_USAGE); + Options options = new Options(); + options.addOption(OPTION_RECORD_NUMBER, true, OPTION_RECORD_NUMBER_DESC); + options.addOption(OPTION_PATH, true, OPTION_PATH_DESC); + options.addOption(OPTION_TABLE, true, OPTION_TABLE_DESC); + options.addOption(OPTION_SET, true, OPTION_SET_DESC); + + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + } + } + + private static class BackupSetCommand extends Command { + private final static String SET_ADD_CMD = "add"; + private final static String SET_REMOVE_CMD = "remove"; + private final static String SET_DELETE_CMD = "delete"; + private final static String SET_DESCRIBE_CMD = "describe"; + private final static String SET_LIST_CMD = "list"; + + BackupSetCommand(Configuration conf, CommandLine cmdline) { + super(conf); + this.cmdline = cmdline; + } + + @Override + public void execute() throws IOException { + super.execute(); + // Command-line must have at least one element + if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String[] args = cmdline.getArgs(); + String cmdStr = args[1]; + BackupCommand cmd = getCommand(cmdStr); + + switch (cmd) { + case SET_ADD: + processSetAdd(args); + break; + case SET_REMOVE: + processSetRemove(args); + break; + case SET_DELETE: + processSetDelete(args); + break; + case SET_DESCRIBE: + processSetDescribe(args); + break; + case SET_LIST: + processSetList(args); + break; + default: + break; + + } + } + + private void processSetList(String[] args) throws IOException { + // List all backup set names + // does not expect any args + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + BackupAdminImpl admin = new BackupAdminImpl(conn);) { + List list = admin.listBackupSets(); + for (BackupSet bs : list) { + System.out.println(bs); + } + } + } + + private void processSetDescribe(String[] args) throws IOException { + if (args == null || args.length != 3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String setName = args[2]; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupSystemTable sysTable = new BackupSystemTable(conn);) { + List tables = sysTable.describeBackupSet(setName); + BackupSet set = tables == null ? null : new BackupSet(setName, tables); + if (set == null) { + System.out.println("Set '" + setName + "' does not exist."); + } else { + System.out.println(set); + } + } + } + + private void processSetDelete(String[] args) throws IOException { + if (args == null || args.length != 3) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String setName = args[2]; + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupAdminImpl admin = new BackupAdminImpl(conn);) { + boolean result = admin.deleteBackupSet(setName); + if (result) { + System.out.println("Delete set " + setName + " OK."); + } else { + System.out.println("Set " + setName + " does not exist"); + } + } + } + + private void processSetRemove(String[] args) throws IOException { + if (args == null || args.length != 4) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + String setName = args[2]; + String[] tables = args[3].split(","); + TableName[] tableNames = toTableNames(tables); + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupAdminImpl admin = new BackupAdminImpl(conn);) { + admin.removeFromBackupSet(setName, tableNames); + } + } + + private TableName[] toTableNames(String[] tables) { + TableName[] arr = new TableName[tables.length]; + for (int i=0; i < tables.length; i++) { + arr[i] = TableName.valueOf(tables[i]); + } + return arr; + } + + private void processSetAdd(String[] args) throws IOException { + if (args == null || args.length != 4) { + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + String setName = args[2]; + String[] tables = args[3].split(","); + TableName[] tableNames = new TableName[tables.length]; + for (int i = 0; i < tables.length; i++) { + tableNames[i] = TableName.valueOf(tables[i]); + } + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + try (final Connection conn = ConnectionFactory.createConnection(conf); + final BackupAdminImpl admin = new BackupAdminImpl(conn);) { + admin.addToBackupSet(setName, tableNames); + } + + } + + private BackupCommand getCommand(String cmdStr) throws IOException { + if (cmdStr.equals(SET_ADD_CMD)) { + return BackupCommand.SET_ADD; + } else if (cmdStr.equals(SET_REMOVE_CMD)) { + return BackupCommand.SET_REMOVE; + } else if (cmdStr.equals(SET_DELETE_CMD)) { + return BackupCommand.SET_DELETE; + } else if (cmdStr.equals(SET_DESCRIBE_CMD)) { + return BackupCommand.SET_DESCRIBE; + } else if (cmdStr.equals(SET_LIST_CMD)) { + return BackupCommand.SET_LIST; + } else { + System.out.println("ERROR: Unknown command for 'set' :" + cmdStr); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + } + + @Override + protected void printUsage() { + System.out.println(SET_CMD_USAGE); + } + + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java new file mode 100644 index 00000000000..2c7d35f7e04 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Backup exception + */ +@SuppressWarnings("serial") +@InterfaceAudience.Private +public class BackupException extends HBaseIOException { + private BackupInfo info; + + /** + * Some exception happened for a backup and don't even know the backup that it was about + * @param msg Full description of the failure + */ + public BackupException(String msg) { + super(msg); + } + + /** + * Some exception happened for a backup with a cause + * @param cause the cause + */ + public BackupException(Throwable cause) { + super(cause); + } + + /** + * Exception for the given backup that has no previous root cause + * @param msg reason why the backup failed + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, BackupInfo desc) { + super(msg); + this.info = desc; + } + + /** + * Exception for the given backup due to another exception + * @param msg reason why the backup failed + * @param cause root cause of the failure + * @param desc description of the backup that is being failed + */ + public BackupException(String msg, Throwable cause, BackupInfo desc) { + super(msg, cause); + this.info = desc; + } + + /** + * Exception when the description of the backup cannot be determined, due to some other root + * cause + * @param message description of what caused the failure + * @param e root cause + */ + public BackupException(String message, Exception e) { + super(message, e); + } + + public BackupInfo getBackupInfo() { + return this.info; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java new file mode 100644 index 00000000000..1d27e79c3f8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -0,0 +1,472 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.master.BackupLogCleaner; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.procedure.ProcedureManagerHost; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Handles backup requests, creates backup info records in backup system table to + * keep track of backup sessions, dispatches backup request. + */ +@InterfaceAudience.Private +public class BackupManager implements Closeable { + private static final Log LOG = LogFactory.getLog(BackupManager.class); + + protected Configuration conf = null; + protected BackupInfo backupInfo = null; + protected BackupSystemTable systemTable; + protected final Connection conn; + + /** + * Backup manager constructor. + * @param conn connection + * @param conf configuration + * @throws IOException exception + */ + public BackupManager(Connection conn, Configuration conf) throws IOException { + if (!conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.BACKUP_ENABLE_DEFAULT)) { + throw new BackupException("HBase backup is not enabled. Check your " + + BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting."); + } + this.conf = conf; + this.conn = conn; + this.systemTable = new BackupSystemTable(conn); + + } + + /** + * Returns backup info + */ + protected BackupInfo getBackupInfo() { + return backupInfo; + } + + /** + * This method modifies the master's configuration in order to inject backup-related features + * (TESTs only) + * @param conf configuration + */ + @VisibleForTesting + public static void decorateMasterConfiguration(Configuration conf) { + if (!isBackupEnabled(conf)) { + return; + } + // Add WAL archive cleaner plug-in + String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS); + String cleanerClass = BackupLogCleaner.class.getCanonicalName(); + if (!plugins.contains(cleanerClass)) { + conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass); + } + + String classes = conf.get(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY); + String masterProcedureClass = LogRollMasterProcedureManager.class.getName(); + if (classes == null) { + conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, masterProcedureClass); + } else if (!classes.contains(masterProcedureClass)) { + conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, classes + "," + masterProcedureClass); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Added log cleaner: " + cleanerClass +"\n" + + "Added master procedure manager: " + masterProcedureClass); + } + + } + + /** + * This method modifies the Region Server configuration in order to inject backup-related features + * TESTs only. + * @param conf configuration + */ + @VisibleForTesting + public static void decorateRegionServerConfiguration(Configuration conf) { + if (!isBackupEnabled(conf)) { + return; + } + + String classes = conf.get(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY); + String regionProcedureClass = LogRollRegionServerProcedureManager.class.getName(); + if (classes == null) { + conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, regionProcedureClass); + } else if (!classes.contains(regionProcedureClass)) { + conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, classes + "," + + regionProcedureClass); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Added region procedure manager: " + regionProcedureClass); + } + + } + + public static boolean isBackupEnabled(Configuration conf) { + return conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.BACKUP_ENABLE_DEFAULT); + } + + /** + * Get configuration + * @return configuration + */ + Configuration getConf() { + return conf; + } + + /** + * Stop all the work of backup. + */ + @Override + public void close() { + + if (systemTable != null) { + try { + systemTable.close(); + } catch (Exception e) { + LOG.error(e); + } + } + } + + /** + * Creates a backup info based on input backup request. + * @param backupId backup id + * @param type type + * @param tableList table list + * @param targetRootDir root dir + * @param workers number of parallel workers + * @param bandwidth bandwidth per worker in MB per sec + * @return BackupInfo + * @throws BackupException exception + */ + public BackupInfo createBackupInfo(String backupId, BackupType type, + List tableList, String targetRootDir, int workers, long bandwidth) + throws BackupException { + if (targetRootDir == null) { + throw new BackupException("Wrong backup request parameter: target backup root directory"); + } + + if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) { + // If table list is null for full backup, which means backup all tables. Then fill the table + // list with all user tables from meta. It no table available, throw the request exception. + + HTableDescriptor[] htds = null; + try (Admin admin = conn.getAdmin()) { + htds = admin.listTables(); + } catch (Exception e) { + throw new BackupException(e); + } + + if (htds == null) { + throw new BackupException("No table exists for full backup of all tables."); + } else { + tableList = new ArrayList<>(); + for (HTableDescriptor hTableDescriptor : htds) { + TableName tn = hTableDescriptor.getTableName(); + if (tn.equals(BackupSystemTable.getTableName(conf))) { + // skip backup system table + continue; + } + tableList.add(hTableDescriptor.getTableName()); + } + + LOG.info("Full backup all the tables available in the cluster: " + tableList); + } + } + + // there are one or more tables in the table list + backupInfo = + new BackupInfo(backupId, type, tableList.toArray(new TableName[tableList.size()]), + targetRootDir); + backupInfo.setBandwidth(bandwidth); + backupInfo.setWorkers(workers); + return backupInfo; + } + + /** + * Check if any ongoing backup. Currently, we only reply on checking status in backup system + * table. We need to consider to handle the case of orphan records in the future. Otherwise, all + * the coming request will fail. + * @return the ongoing backup id if on going backup exists, otherwise null + * @throws IOException exception + */ + private String getOngoingBackupId() throws IOException { + + ArrayList sessions = systemTable.getBackupInfos(BackupState.RUNNING); + if (sessions.size() == 0) { + return null; + } + return sessions.get(0).getBackupId(); + } + + /** + * Start the backup manager service. + * @throws IOException exception + */ + public void initialize() throws IOException { + String ongoingBackupId = this.getOngoingBackupId(); + if (ongoingBackupId != null) { + LOG.info("There is a ongoing backup " + ongoingBackupId + + ". Can not launch new backup until no ongoing backup remains."); + throw new BackupException("There is ongoing backup."); + } + } + + public void setBackupInfo(BackupInfo backupInfo) { + this.backupInfo = backupInfo; + } + + /** + * Get direct ancestors of the current backup. + * @param backupInfo The backup info for the current backup + * @return The ancestors for the current backup + * @throws IOException exception + * @throws BackupException exception + */ + public ArrayList getAncestors(BackupInfo backupInfo) throws IOException, + BackupException { + LOG.debug("Getting the direct ancestors of the current backup " + backupInfo.getBackupId()); + + ArrayList ancestors = new ArrayList(); + + // full backup does not have ancestor + if (backupInfo.getType() == BackupType.FULL) { + LOG.debug("Current backup is a full backup, no direct ancestor for it."); + return ancestors; + } + + // get all backup history list in descending order + + ArrayList allHistoryList = getBackupHistory(true); + for (BackupInfo backup : allHistoryList) { + + BackupImage.Builder builder = BackupImage.newBuilder(); + + BackupImage image = + builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames()) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + + // add the full backup image as an ancestor until the last incremental backup + if (backup.getType().equals(BackupType.FULL)) { + // check the backup image coverage, if previous image could be covered by the newer ones, + // then no need to add + if (!BackupManifest.canCoverImage(ancestors, image)) { + ancestors.add(image); + } + } else { + // found last incremental backup, if previously added full backup ancestor images can cover + // it, then this incremental ancestor is not the dependent of the current incremental + // backup, that is to say, this is the backup scope boundary of current table set. + // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing + // incremental backup + if (BackupManifest.canCoverImage(ancestors, image)) { + LOG.debug("Met the backup boundary of the current table set:"); + for (BackupImage image1 : ancestors) { + LOG.debug(" BackupID=" + image1.getBackupId() + ", BackupDir=" + image1.getRootDir()); + } + } else { + Path logBackupPath = + HBackupFileSystem.getLogBackupPath(backup.getBackupRootDir(), backup.getBackupId()); + 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: " + "{BackupID=" + + lastIncrImage.getBackupId() + "," + "BackupDir=" + 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 backupInfo backup info + * @param table table + * @return backupImages on the dependency list + * @throws BackupException exception + * @throws IOException exception + */ + public ArrayList getAncestors(BackupInfo backupInfo, TableName table) + throws BackupException, IOException { + ArrayList ancestors = getAncestors(backupInfo); + ArrayList tableAncestors = new ArrayList(); + for (BackupImage image : ancestors) { + if (image.hasTable(table)) { + tableAncestors.add(image); + if (image.getType() == BackupType.FULL) { + break; + } + } + } + return tableAncestors; + } + + /* + * backup system table operations + */ + + /** + * Updates status (state) of a backup session in a persistent store + * @param context context + * @throws IOException exception + */ + public void updateBackupInfo(BackupInfo context) throws IOException { + systemTable.updateBackupInfo(context); + } + + /** + * Read the last backup start code (timestamp) of last successful backup. Will return null if + * there is no startcode stored in backup system table or the value is of length 0. These two + * cases indicate there is no successful backup completed so far. + * @return the timestamp of a last successful backup + * @throws IOException exception + */ + public String readBackupStartCode() throws IOException { + return systemTable.readBackupStartCode(backupInfo.getBackupRootDir()); + } + + /** + * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte. + * @param startCode start code + * @throws IOException exception + */ + public void writeBackupStartCode(Long startCode) throws IOException { + systemTable.writeBackupStartCode(startCode, backupInfo.getBackupRootDir()); + } + + /** + * Get the RS log information after the last log roll from backup system table. + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult() throws IOException { + return systemTable.readRegionServerLastLogRollResult(backupInfo.getBackupRootDir()); + } + + /** + * Get all completed backup information (in desc order by time) + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public List getBackupHistory() throws IOException { + return systemTable.getBackupHistory(); + } + + public ArrayList getBackupHistory(boolean completed) throws IOException { + return systemTable.getBackupHistory(completed); + } + + /** + * Write the current timestamps for each regionserver to backup system table after a successful + * full or incremental backup. Each table may have a different set of log timestamps. The saved + * timestamp is of the last log file that was backed up already. + * @param tables tables + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, + HashMap newTimestamps) throws IOException { + systemTable.writeRegionServerLogTimestamp(tables, newTimestamps, + backupInfo.getBackupRootDir()); + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public HashMap> readLogTimestampMap() throws IOException { + return systemTable.readLogTimestampMap(backupInfo.getBackupRootDir()); + } + + /** + * Return the current tables covered by incremental backup. + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet() throws IOException { + return systemTable.getIncrementalBackupTableSet(backupInfo.getBackupRootDir()); + } + + /** + * Adds set of tables to overall incremental backup table set + * @param tables tables + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables) throws IOException { + systemTable.addIncrementalBackupTableSet(tables, backupInfo.getBackupRootDir()); + } + + /** + * 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, backupInfo.getBackupId(), backupInfo.getBackupRootDir()); + } + + /** + * Get WAL files iterator + * @return WAL files iterator from backup system table + * @throws IOException + */ + public Iterator getWALFilesFromBackupSystem() throws IOException { + return systemTable.getWALFilesIterator(backupInfo.getBackupRootDir()); + } + + public Connection getConnection() { + return conn; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java new file mode 100644 index 00000000000..b8adac9fba8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -0,0 +1,666 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + +/** + * Backup manifest contains all the meta data of a backup image. The manifest info will be bundled + * as manifest file together with data. So that each backup image will contain all the info needed + * for restore. BackupManifest is a storage container for BackupImage. + * It is responsible for storing/reading backup image data and has some additional utility methods. + * + */ +@InterfaceAudience.Private +public class BackupManifest { + + private static final Log LOG = LogFactory.getLog(BackupManifest.class); + + // manifest file name + public static final String MANIFEST_FILE_NAME = ".backup.manifest"; + + /** + * Backup image, the dependency graph is made up by series of backup images + * BackupImage contains all the relevant information to restore the backup and + * is used during restore operation + */ + + public static class BackupImage implements Comparable { + + static class Builder { + BackupImage image; + + Builder() { + image = new BackupImage(); + } + + Builder withBackupId(String backupId) { + image.setBackupId(backupId); + return this; + } + + Builder withType(BackupType type) { + image.setType(type); + return this; + } + + Builder withRootDir(String rootDir) { + image.setRootDir(rootDir); + return this; + } + + Builder withTableList(List tableList) { + image.setTableList(tableList); + return this; + } + + Builder withStartTime(long startTime) { + image.setStartTs(startTime); + return this; + } + + Builder withCompleteTime(long completeTime) { + image.setCompleteTs(completeTime); + return this; + } + + BackupImage build() { + return image; + } + + } + + private String backupId; + private BackupType type; + private String rootDir; + private List tableList; + private long startTs; + private long completeTs; + private ArrayList ancestors; + private HashMap> incrTimeRanges; + + static Builder newBuilder() { + return new Builder(); + } + + public BackupImage() { + super(); + } + + private BackupImage(String backupId, BackupType type, String rootDir, + List tableList, long startTs, long completeTs) { + this.backupId = backupId; + this.type = type; + this.rootDir = rootDir; + this.tableList = tableList; + this.startTs = startTs; + this.completeTs = completeTs; + } + + static BackupImage fromProto(BackupProtos.BackupImage im) { + String backupId = im.getBackupId(); + String rootDir = im.getBackupRootDir(); + long startTs = im.getStartTs(); + long completeTs = im.getCompleteTs(); + List tableListList = im.getTableListList(); + List tableList = new ArrayList(); + for (HBaseProtos.TableName tn : tableListList) { + tableList.add(ProtobufUtil.toTableName(tn)); + } + + List ancestorList = im.getAncestorsList(); + + BackupType type = + im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL + : BackupType.INCREMENTAL; + + BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); + for (BackupProtos.BackupImage img : ancestorList) { + image.addAncestor(fromProto(img)); + } + image.setIncrTimeRanges(loadIncrementalTimestampMap(im)); + return image; + } + + BackupProtos.BackupImage toProto() { + BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder(); + builder.setBackupId(backupId); + builder.setCompleteTs(completeTs); + builder.setStartTs(startTs); + builder.setBackupRootDir(rootDir); + if (type == BackupType.FULL) { + builder.setBackupType(BackupProtos.BackupType.FULL); + } else { + builder.setBackupType(BackupProtos.BackupType.INCREMENTAL); + } + + for (TableName name : tableList) { + builder.addTableList(ProtobufUtil.toProtoTableName(name)); + } + + if (ancestors != null) { + for (BackupImage im : ancestors) { + builder.addAncestors(im.toProto()); + } + } + + setIncrementalTimestampMap(builder); + return builder.build(); + } + + private static HashMap> loadIncrementalTimestampMap( + BackupProtos.BackupImage proto) { + List list = proto.getTstMapList(); + + HashMap> incrTimeRanges = + new HashMap>(); + if (list == null || list.size() == 0) return incrTimeRanges; + for (BackupProtos.TableServerTimestamp tst : list) { + TableName tn = ProtobufUtil.toTableName(tst.getTableName()); + HashMap map = incrTimeRanges.get(tn); + if (map == null) { + map = new HashMap(); + incrTimeRanges.put(tn, map); + } + List listSt = tst.getServerTimestampList(); + for (BackupProtos.ServerTimestamp stm : listSt) { + ServerName sn = ProtobufUtil.toServerName(stm.getServerName()); + map.put(sn.getHostname() + ":" + sn.getPort(), stm.getTimestamp()); + } + } + return incrTimeRanges; + } + + private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) { + if (this.incrTimeRanges == null) { + return; + } + for (Entry> entry : this.incrTimeRanges.entrySet()) { + TableName key = entry.getKey(); + HashMap value = entry.getValue(); + BackupProtos.TableServerTimestamp.Builder tstBuilder = + BackupProtos.TableServerTimestamp.newBuilder(); + tstBuilder.setTableName(ProtobufUtil.toProtoTableName(key)); + + for (Map.Entry entry2 : value.entrySet()) { + String s = entry2.getKey(); + BackupProtos.ServerTimestamp.Builder stBuilder = + BackupProtos.ServerTimestamp.newBuilder(); + HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder(); + ServerName sn = ServerName.parseServerName(s); + snBuilder.setHostName(sn.getHostname()); + snBuilder.setPort(sn.getPort()); + stBuilder.setServerName(snBuilder.build()); + stBuilder.setTimestamp(entry2.getValue()); + tstBuilder.addServerTimestamp(stBuilder.build()); + } + builder.addTstMap(tstBuilder.build()); + } + } + + public String getBackupId() { + return backupId; + } + + private void setBackupId(String backupId) { + this.backupId = backupId; + } + + public BackupType getType() { + return type; + } + + private void setType(BackupType type) { + this.type = type; + } + + public String getRootDir() { + return rootDir; + } + + private void setRootDir(String rootDir) { + this.rootDir = rootDir; + } + + public List getTableNames() { + return tableList; + } + + private void setTableList(List tableList) { + this.tableList = tableList; + } + + public long getStartTs() { + return startTs; + } + + private void setStartTs(long startTs) { + this.startTs = startTs; + } + + public long getCompleteTs() { + return completeTs; + } + + private void setCompleteTs(long completeTs) { + this.completeTs = completeTs; + } + + public ArrayList getAncestors() { + if (this.ancestors == null) { + this.ancestors = new ArrayList(); + } + return this.ancestors; + } + + private void addAncestor(BackupImage backupImage) { + this.getAncestors().add(backupImage); + } + + public boolean hasAncestor(String token) { + for (BackupImage image : this.getAncestors()) { + if (image.getBackupId().equals(token)) { + return true; + } + } + return false; + } + + public boolean hasTable(TableName table) { + return tableList.contains(table); + } + + @Override + public int compareTo(BackupImage other) { + String thisBackupId = this.getBackupId(); + String otherBackupId = other.getBackupId(); + int index1 = thisBackupId.lastIndexOf("_"); + int index2 = otherBackupId.lastIndexOf("_"); + String name1 = thisBackupId.substring(0, index1); + String name2 = otherBackupId.substring(0, index2); + if (name1.equals(name2)) { + Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1)); + Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1)); + return thisTS.compareTo(otherTS); + } else { + return name1.compareTo(name2); + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BackupImage) { + return this.compareTo((BackupImage) obj) == 0; + } + return false; + } + + @Override + public int hashCode() { + int hash = 33 * this.getBackupId().hashCode() + type.hashCode(); + hash = 33 * hash + rootDir.hashCode(); + hash = 33 * hash + Long.valueOf(startTs).hashCode(); + hash = 33 * hash + Long.valueOf(completeTs).hashCode(); + for (TableName table : tableList) { + hash = 33 * hash + table.hashCode(); + } + return hash; + } + + public HashMap> getIncrTimeRanges() { + return incrTimeRanges; + } + + private void setIncrTimeRanges(HashMap> incrTimeRanges) { + this.incrTimeRanges = incrTimeRanges; + } + } + + // backup image directory + private String tableBackupDir = null; + private BackupImage backupImage; + + /** + * Construct manifest for a ongoing backup. + * @param backup The ongoing backup info + */ + public BackupManifest(BackupInfo backup) { + + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = + builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames()) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + } + + /** + * Construct a table level manifest for a backup of the named table. + * @param backup The ongoing backup session info + */ + public BackupManifest(BackupInfo backup, TableName table) { + this.tableBackupDir = backup.getTableBackupDir(table); + List tables = new ArrayList(); + tables.add(table); + BackupImage.Builder builder = BackupImage.newBuilder(); + this.backupImage = + builder.withBackupId(backup.getBackupId()).withType(backup.getType()) + .withRootDir(backup.getBackupRootDir()).withTableList(tables) + .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build(); + } + + /** + * Construct manifest from a backup directory. + * @param conf configuration + * @param backupPath backup path + * @throws IOException + */ + + public BackupManifest(Configuration conf, Path backupPath) throws IOException { + this(backupPath.getFileSystem(conf), backupPath); + } + + /** + * Construct manifest from a backup directory. + * @param fs the FileSystem + * @param backupPath backup path + * @throws BackupException exception + */ + + public BackupManifest(FileSystem fs, Path backupPath) throws BackupException { + if (LOG.isDebugEnabled()) { + LOG.debug("Loading manifest from: " + backupPath.toString()); + } + // The input backupDir may not exactly be the backup table dir. + // It could be the backup log dir where there is also a manifest file stored. + // This variable's purpose is to keep the correct and original location so + // that we can store/persist it. + try { + + FileStatus[] subFiles = BackupUtils.listStatus(fs, backupPath, null); + if (subFiles == null) { + String errorMsg = backupPath.toString() + " does not exist"; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + for (FileStatus subFile : subFiles) { + if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) { + + // load and set manifest field from file content + FSDataInputStream in = fs.open(subFile.getPath()); + long len = subFile.getLen(); + byte[] pbBytes = new byte[(int) len]; + in.readFully(pbBytes); + BackupProtos.BackupImage proto = null; + try { + proto = BackupProtos.BackupImage.parseFrom(pbBytes); + } catch (Exception e) { + throw new BackupException(e); + } + this.backupImage = BackupImage.fromProto(proto); + LOG.debug("Loaded manifest instance from manifest file: " + + BackupUtils.getPath(subFile.getPath())); + return; + } + } + String errorMsg = "No manifest file found in: " + backupPath.toString(); + throw new IOException(errorMsg); + + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + } + + public BackupType getType() { + return backupImage.getType(); + } + + /** + * Get the table set of this image. + * @return The table set list + */ + public List getTableList() { + return backupImage.getTableNames(); + } + + /** + * Persist the manifest file. + * @throws IOException IOException when storing the manifest file. + */ + + public void store(Configuration conf) throws BackupException { + byte[] data = backupImage.toProto().toByteArray(); + // write the file, overwrite if already exist + String logBackupDir = + BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId()); + Path manifestFilePath = + new Path(new Path((tableBackupDir != null ? tableBackupDir : logBackupDir)), + MANIFEST_FILE_NAME); + try (FSDataOutputStream out = + manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);) { + out.write(data); + } catch (IOException e) { + throw new BackupException(e.getMessage()); + } + + LOG.info("Manifest file stored to " + manifestFilePath); + } + + /** + * Get this backup image. + * @return the backup image. + */ + public BackupImage getBackupImage() { + return backupImage; + } + + /** + * Add dependent backup image for this backup. + * @param image The direct dependent backup image + */ + public void addDependentImage(BackupImage image) { + this.backupImage.addAncestor(image); + } + + /** + * Set the incremental timestamp map directly. + * @param incrTimestampMap timestamp map + */ + public void setIncrTimestampMap(HashMap> incrTimestampMap) { + this.backupImage.setIncrTimeRanges(incrTimestampMap); + } + + public Map> getIncrTimestampMap() { + return backupImage.getIncrTimeRanges(); + } + + /** + * Get the image list of this backup for restore in time order. + * @param reverse If true, then output in reverse order, otherwise in time order from old to new + * @return the backup image list for restore in time order + */ + public ArrayList getRestoreDependentList(boolean reverse) { + TreeMap restoreImages = new TreeMap(); + restoreImages.put(backupImage.startTs, backupImage); + for (BackupImage image : backupImage.getAncestors()) { + restoreImages.put(Long.valueOf(image.startTs), image); + } + return new ArrayList(reverse ? (restoreImages.descendingMap().values()) + : (restoreImages.values())); + } + + /** + * Get the dependent image list for a specific table of this backup in time order from old to new + * if want to restore to this backup image level. + * @param table table + * @return the backup image list for a table in time order + */ + public ArrayList getDependentListByTable(TableName table) { + ArrayList tableImageList = new ArrayList(); + ArrayList imageList = getRestoreDependentList(true); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + if (image.getType() == BackupType.FULL) { + break; + } + } + } + Collections.reverse(tableImageList); + return tableImageList; + } + + /** + * Get the full dependent image list in the whole dependency scope for a specific table of this + * backup in time order from old to new. + * @param table table + * @return the full backup image list for a table in time order in the whole scope of the + * dependency of this image + */ + public ArrayList getAllDependentListByTable(TableName table) { + ArrayList tableImageList = new ArrayList(); + ArrayList imageList = getRestoreDependentList(false); + for (BackupImage image : imageList) { + if (image.hasTable(table)) { + tableImageList.add(image); + } + } + return tableImageList; + } + + /** + * Check whether backup image1 could cover backup image2 or not. + * @param image1 backup image 1 + * @param image2 backup image 2 + * @return true if image1 can cover image2, otherwise false + */ + public static boolean canCoverImage(BackupImage image1, BackupImage image2) { + // image1 can cover image2 only when the following conditions are satisfied: + // - image1 must not be an incremental image; + // - image1 must be taken after image2 has been taken; + // - table set of image1 must cover the table set of image2. + if (image1.getType() == BackupType.INCREMENTAL) { + return false; + } + if (image1.getStartTs() < image2.getStartTs()) { + return false; + } + List image1TableList = image1.getTableNames(); + List image2TableList = image2.getTableNames(); + boolean found = false; + for (int i = 0; i < image2TableList.size(); i++) { + found = false; + for (int j = 0; j < image1TableList.size(); j++) { + if (image2TableList.get(i).equals(image1TableList.get(j))) { + found = true; + break; + } + } + if (!found) { + return false; + } + } + + LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId()); + return true; + } + + /** + * Check whether backup image set could cover a backup image or not. + * @param fullImages The backup image set + * @param image The target backup image + * @return true if fullImages can cover image, otherwise false + */ + public static boolean canCoverImage(ArrayList fullImages, BackupImage image) { + // fullImages can cover image only when the following conditions are satisfied: + // - each image of fullImages must not be an incremental image; + // - each image of fullImages must be taken after image has been taken; + // - sum table set of fullImages must cover the table set of image. + for (BackupImage image1 : fullImages) { + if (image1.getType() == BackupType.INCREMENTAL) { + return false; + } + if (image1.getStartTs() < image.getStartTs()) { + return false; + } + } + + ArrayList image1TableList = new ArrayList(); + for (BackupImage image1 : fullImages) { + List tableList = image1.getTableNames(); + for (TableName table : tableList) { + image1TableList.add(table.getNameAsString()); + } + } + ArrayList image2TableList = new ArrayList(); + List tableList = image.getTableNames(); + for (TableName table : tableList) { + image2TableList.add(table.getNameAsString()); + } + + for (int i = 0; i < image2TableList.size(); i++) { + if (image1TableList.contains(image2TableList.get(i)) == false) { + return false; + } + } + + LOG.debug("Full image set can cover image " + image.getBackupId()); + return true; + } + + public BackupInfo toBackupInfo() { + BackupInfo info = new BackupInfo(); + info.setType(backupImage.getType()); + List list = backupImage.getTableNames(); + TableName[] tables = new TableName[list.size()]; + info.addTables(list.toArray(tables)); + info.setBackupId(backupImage.getBackupId()); + info.setStartTs(backupImage.getStartTs()); + info.setBackupRootDir(backupImage.getRootDir()); + if (backupImage.getType() == BackupType.INCREMENTAL) { + info.setHLogTargetDir(BackupUtils.getLogBackupDir(backupImage.getRootDir(), + backupImage.getBackupId())); + } + return info; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java new file mode 100644 index 00000000000..6362f8e0a77 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -0,0 +1,1376 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +/** + * This class provides API to access backup system table
+ * + * Backup system table schema:
+ *

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

+ */ +@InterfaceAudience.Private +public final class BackupSystemTable implements Closeable { + + static class WALItem { + String backupId; + String walFile; + String backupRoot; + + WALItem(String backupId, String walFile, String backupRoot) { + this.backupId = backupId; + this.walFile = walFile; + this.backupRoot = backupRoot; + } + + public String getBackupId() { + return backupId; + } + + public String getWalFile() { + return walFile; + } + + public String getBackupRoot() { + return backupRoot; + } + + @Override + public String toString() { + return Path.SEPARATOR + backupRoot + Path.SEPARATOR + backupId + Path.SEPARATOR + walFile; + } + + } + + private static final Log LOG = LogFactory.getLog(BackupSystemTable.class); + + private TableName tableName; + /** + * Stores backup sessions (contexts) + */ + final static byte[] SESSIONS_FAMILY = "session".getBytes(); + /** + * Stores other meta + */ + final static byte[] META_FAMILY = "meta".getBytes(); + /** + * Connection to HBase cluster, shared among all instances + */ + private final Connection connection; + + + private final static String BACKUP_INFO_PREFIX = "session:"; + private final static String START_CODE_ROW = "startcode:"; + private final static 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 String SET_KEY_PREFIX = "backupset:"; + + private final static byte[] EMPTY_VALUE = new byte[] {}; + + // Safe delimiter in a string + private final static String NULL = "\u0000"; + + public BackupSystemTable(Connection conn) throws IOException { + this.connection = conn; + tableName = BackupSystemTable.getTableName(conn.getConfiguration()); + checkSystemTable(); + } + + private void checkSystemTable() throws IOException { + try (Admin admin = connection.getAdmin();) { + + if (!admin.tableExists(tableName)) { + HTableDescriptor backupHTD = + BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration()); + admin.createTable(backupHTD); + } + waitForSystemTable(admin); + } + } + + private void waitForSystemTable(Admin admin) throws IOException { + long TIMEOUT = 60000; + long startTime = EnvironmentEdgeManager.currentTime(); + while (!admin.tableExists(tableName) || !admin.isTableAvailable(tableName)) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) { + throw new IOException("Failed to create backup system table after "+ TIMEOUT+"ms"); + } + } + LOG.debug("Backup table exists and available"); + + } + + + + @Override + public void close() { + // do nothing + } + + /** + * Updates status (state) of a backup session in backup system table table + * @param info backup info + * @throws IOException exception + */ + public void updateBackupInfo(BackupInfo info) throws IOException { + + if (LOG.isTraceEnabled()) { + LOG.trace("update backup status in backup system table for: " + info.getBackupId() + + " set status=" + info.getState()); + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForBackupInfo(info); + table.put(put); + } + } + + /** + * Deletes backup status from backup system table table + * @param backupId backup id + * @throws IOException exception + */ + + public void deleteBackupInfo(String backupId) throws IOException { + + if (LOG.isTraceEnabled()) { + LOG.trace("delete backup status in backup system table for " + backupId); + } + try (Table table = connection.getTable(tableName)) { + Delete del = createDeleteForBackupInfo(backupId); + table.delete(del); + } + } + + /** + * Reads backup status object (instance of backup info) from backup system table table + * @param backupId backup id + * @return Current status of backup session or null + */ + + public BackupInfo readBackupInfo(String backupId) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read backup status from backup system table for: " + backupId); + } + + try (Table table = connection.getTable(tableName)) { + Get get = createGetForBackupInfo(backupId); + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + return resultToBackupInfo(res); + } + } + + /** + * Read the last backup start code (timestamp) of last successful backup. Will return null if + * there is no start code stored on hbase or the value is of length 0. These two cases indicate + * there is no successful backup completed so far. + * @param backupRoot directory path to backup destination + * @return the timestamp of last successful backup + * @throws IOException exception + */ + public String readBackupStartCode(String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read backup start code from backup system table"); + } + try (Table table = connection.getTable(tableName)) { + Get get = createGetForStartCode(backupRoot); + Result res = table.get(get); + if (res.isEmpty()) { + return null; + } + Cell cell = res.listCells().get(0); + byte[] val = CellUtil.cloneValue(cell); + if (val.length == 0) { + return null; + } + return new String(val); + } + } + + /** + * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte. + * @param startCode start code + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("write backup start code to backup system table " + startCode); + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForStartCode(startCode.toString(), backupRoot); + table.put(put); + } + } + + /** + * Get the Region Servers log information after the last log roll from backup system table. + * @param backupRoot root directory path to backup + * @return RS log info + * @throws IOException exception + */ + public HashMap readRegionServerLastLogRollResult(String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read region server last roll log result to backup system table"); + } + + Scan scan = createScanForReadRegionServerLastLogRollResult(backupRoot); + + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res = null; + HashMap rsTimestampMap = new HashMap(); + while ((res = scanner.next()) != null) { + res.advance(); + Cell cell = res.current(); + byte[] row = CellUtil.cloneRow(cell); + String server = + getServerNameForReadRegionServerLastLogRollResult(row); + byte[] data = CellUtil.cloneValue(cell); + rsTimestampMap.put(server, Bytes.toLong(data)); + } + return rsTimestampMap; + } + } + + /** + * Writes Region Server last roll log result (timestamp) to backup system table table + * @param server Region Server name + * @param ts last log timestamp + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("write region server last roll log result to backup system table"); + } + try (Table table = connection.getTable(tableName)) { + Put put = + createPutForRegionServerLastLogRollResult(server, ts, backupRoot); + table.put(put); + } + } + + /** + * Get all completed backup information (in desc order by time) + * @param onlyCompleted true, if only successfully completed sessions + * @return history info of BackupCompleteData + * @throws IOException exception + */ + public ArrayList getBackupHistory(boolean onlyCompleted) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("get backup history from backup system table"); + } + ArrayList list; + BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY; + list = getBackupInfos(state); + return BackupUtils.sortHistoryListDesc(list); + } + + /** + * Get all backups history + * @return list of backup info + * @throws IOException + */ + public List getBackupHistory() throws IOException { + return getBackupHistory(false); + } + + /** + * Get first n backup history records + * @param n number of records + * @return list of records + * @throws IOException + */ + public List getHistory(int n) throws IOException { + + List history = getBackupHistory(); + if (history.size() <= n) return history; + List list = new ArrayList(); + for (int i = 0; i < n; i++) { + list.add(history.get(i)); + } + return list; + + } + + /** + * Get backup history records filtered by list of filters. + * @param n max number of records + * @param filters list of filters + * @return backup records + * @throws IOException + */ + public List getBackupHistory(int n, BackupInfo.Filter... filters) throws IOException { + if (filters.length == 0) return getHistory(n); + + List history = getBackupHistory(); + List result = new ArrayList(); + for (BackupInfo bi : history) { + if (result.size() == n) break; + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(bi)) { + passed = false; + break; + } + } + if (passed) { + result.add(bi); + } + } + return result; + + } + + /** + * Get history for backup destination + * @param backupRoot backup destination path + * @return List of backup info + * @throws IOException + */ + public List getBackupHistory(String backupRoot) throws IOException { + ArrayList history = getBackupHistory(false); + for (Iterator iterator = history.iterator(); iterator.hasNext();) { + BackupInfo info = iterator.next(); + if (!backupRoot.equals(info.getBackupRootDir())) { + iterator.remove(); + } + } + return history; + } + + /** + * Get history for a table + * @param name table name + * @return history for a table + * @throws IOException + */ + public List getBackupHistoryForTable(TableName name) throws IOException { + List history = getBackupHistory(); + List tableHistory = new ArrayList(); + for (BackupInfo info : history) { + List tables = info.getTableNames(); + if (tables.contains(name)) { + tableHistory.add(info); + } + } + return tableHistory; + } + + public Map> getBackupHistoryForTableSet(Set set, + String backupRoot) throws IOException { + List history = getBackupHistory(backupRoot); + Map> tableHistoryMap = + new HashMap>(); + for (Iterator iterator = history.iterator(); iterator.hasNext();) { + BackupInfo info = iterator.next(); + if (!backupRoot.equals(info.getBackupRootDir())) { + continue; + } + List tables = info.getTableNames(); + for (TableName tableName : tables) { + if (set.contains(tableName)) { + ArrayList list = tableHistoryMap.get(tableName); + if (list == null) { + list = new ArrayList(); + tableHistoryMap.put(tableName, list); + } + list.add(info); + } + } + } + return tableHistoryMap; + } + + /** + * Get all backup sessions with a given state (in descending order by time) + * @param state backup session state + * @return history info of backup info objects + * @throws IOException exception + */ + public ArrayList getBackupInfos(BackupState state) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("get backup infos from backup system table"); + } + + Scan scan = createScanForBackupHistory(); + ArrayList list = new ArrayList(); + + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + BackupInfo context = cellToBackupInfo(res.current()); + if (state != BackupState.ANY && context.getState() != state) { + continue; + } + list.add(context); + } + return list; + } + } + + /** + * Write the current timestamps for each regionserver to backup system table after a successful + * full or incremental backup. The saved timestamp is of the last log file that was backed up + * already. + * @param tables tables + * @param newTimestamps timestamps + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void writeRegionServerLogTimestamp(Set tables, + HashMap newTimestamps, String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("write RS log time stamps to backup system table for tables [" + + StringUtils.join(tables, ",") + "]"); + } + List puts = new ArrayList(); + for (TableName table : tables) { + byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray(); + Put put = + createPutForWriteRegionServerLogTimestamp(table, smapData, + backupRoot); + puts.add(put); + } + try (Table table = connection.getTable(tableName)) { + table.put(puts); + } + } + + /** + * Read the timestamp for each region server log after the last successful backup. Each table has + * its own set of the timestamps. The info is stored for each table as a concatenated string of + * rs->timestapmp + * @param backupRoot root directory path to backup + * @return the timestamp for each region server. key: tableName value: + * RegionServer,PreviousTimeStamp + * @throws IOException exception + */ + public HashMap> readLogTimestampMap(String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("read RS log ts from backup system table for root=" + backupRoot); + } + + HashMap> tableTimestampMap = + new HashMap>(); + + Scan scan = createScanForReadLogTimestampMap(backupRoot); + try (Table table = connection.getTable(tableName); + ResultScanner 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 = getTableNameForReadLogTimestampMap(row); + TableName tn = TableName.valueOf(tabName); + byte[] data = CellUtil.cloneValue(cell); + if (data == null) { + throw new IOException("Data of last backup data from backup system table " + + "is empty. Create a backup first."); + } + if (data != null && data.length > 0) { + HashMap lastBackup = + fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data)); + tableTimestampMap.put(tn, lastBackup); + } + } + return tableTimestampMap; + } + } + + private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table, + Map map) { + BackupProtos.TableServerTimestamp.Builder tstBuilder = + BackupProtos.TableServerTimestamp.newBuilder(); + tstBuilder.setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil + .toProtoTableName(table)); + + for (Entry entry : map.entrySet()) { + BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder(); + HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder(); + ServerName sn = ServerName.parseServerName(entry.getKey()); + snBuilder.setHostName(sn.getHostname()); + snBuilder.setPort(sn.getPort()); + builder.setServerName(snBuilder.build()); + builder.setTimestamp(entry.getValue()); + tstBuilder.addServerTimestamp(builder.build()); + } + + return tstBuilder.build(); + } + + private HashMap fromTableServerTimestampProto( + BackupProtos.TableServerTimestamp proto) { + HashMap map = new HashMap(); + List list = proto.getServerTimestampList(); + for (BackupProtos.ServerTimestamp st : list) { + ServerName sn = + org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(st.getServerName()); + map.put(sn.getHostname() + ":" + sn.getPort(), st.getTimestamp()); + } + return map; + } + + /** + * Return the current tables covered by incremental backup. + * @param backupRoot root directory path to backup + * @return set of tableNames + * @throws IOException exception + */ + public Set getIncrementalBackupTableSet(String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("get incremental backup table set from backup system table"); + } + TreeSet set = new TreeSet<>(); + + try (Table table = connection.getTable(tableName)) { + Get get = createGetForIncrBackupTableSet(backupRoot); + Result res = table.get(get); + if (res.isEmpty()) { + return set; + } + List cells = res.listCells(); + for (Cell cell : cells) { + // qualifier = table name - we use table names as qualifiers + set.add(TableName.valueOf(CellUtil.cloneQualifier(cell))); + } + return set; + } + } + + /** + * Add tables to global incremental backup set + * @param tables set of tables + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public void addIncrementalBackupTableSet(Set tables, String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Add incremental backup table set to backup system table. ROOT=" + backupRoot + + " tables [" + StringUtils.join(tables, " ") + "]"); + for (TableName table : tables) { + LOG.debug(table); + } + } + try (Table table = connection.getTable(tableName)) { + Put put = createPutForIncrBackupTableSet(tables, backupRoot); + table.put(put); + } + } + + /** + * Deletes incremental backup set for a backup destination + * @param backupRoot backup root + */ + + public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Delete incremental backup table set to backup system table. ROOT=" + backupRoot); + } + try (Table table = connection.getTable(tableName)) { + Delete delete = createDeleteForIncrBackupTableSet(backupRoot); + table.delete(delete); + } + } + + /** + * Register WAL files as eligible for deletion + * @param files files + * @param backupId backup id + * @param backupRoot root directory path to backup destination + * @throws IOException exception + */ + public void addWALFiles(List files, String backupId, String backupRoot) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("add WAL files to backup system table: " + backupId + " " + backupRoot + " files [" + + StringUtils.join(files, ",") + "]"); + for (String f : files) { + LOG.debug("add :" + f); + } + } + try (Table table = connection.getTable(tableName)) { + List puts = + createPutsForAddWALFiles(files, backupId, backupRoot); + table.put(puts); + } + } + + /** + * Register WAL files as eligible for deletion + * @param backupRoot root directory path to backup + * @throws IOException exception + */ + public Iterator getWALFilesIterator(String backupRoot) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("get WAL files from backup system table"); + } + final Table table = connection.getTable(tableName); + Scan scan = createScanForGetWALs(backupRoot); + final ResultScanner scanner = table.getScanner(scan); + final Iterator it = scanner.iterator(); + return new Iterator() { + + @Override + public boolean hasNext() { + boolean next = it.hasNext(); + if (!next) { + // close all + try { + scanner.close(); + table.close(); + } catch (IOException e) { + LOG.error("Close WAL Iterator", e); + } + } + return next; + } + + @Override + public WALItem next() { + Result next = it.next(); + List cells = next.listCells(); + byte[] buf = cells.get(0).getValueArray(); + int len = cells.get(0).getValueLength(); + int offset = cells.get(0).getValueOffset(); + String backupId = new String(buf, offset, len); + buf = cells.get(1).getValueArray(); + len = cells.get(1).getValueLength(); + offset = cells.get(1).getValueOffset(); + String walFile = new String(buf, offset, len); + buf = cells.get(2).getValueArray(); + len = cells.get(2).getValueLength(); + offset = cells.get(2).getValueOffset(); + String backupRoot = new String(buf, offset, len); + return new WALItem(backupId, walFile, backupRoot); + } + + @Override + public void remove() { + // not implemented + throw new RuntimeException("remove is not supported"); + } + }; + + } + + /** + * Check if WAL file is eligible for deletion Future: to support all backup destinations + * @param file name of a file to check + * @return true, if deletable, false otherwise. + * @throws IOException exception + */ + public boolean isWALFileDeletable(String file) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Check if WAL file has been already backed up in backup system table " + file); + } + try (Table table = connection.getTable(tableName)) { + Get get = createGetForCheckWALFile(file); + Result res = table.get(get); + if (res.isEmpty()) { + return false; + } + return true; + } + } + + /** + * Checks if we have at least one backup session in backup system table This API is used by + * BackupLogCleaner + * @return true, if - at least one session exists in backup system table table + * @throws IOException exception + */ + public boolean hasBackupSessions() throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Has backup sessions from backup system table"); + } + boolean result = false; + Scan scan = createScanForBackupHistory(); + scan.setCaching(1); + try (Table table = connection.getTable(tableName); + ResultScanner scanner = table.getScanner(scan)) { + if (scanner.next() != null) { + result = true; + } + return result; + } + } + + /** + * BACKUP SETS + */ + + /** + * Get backup set list + * @return backup set list + * @throws IOException + */ + public List listBackupSets() throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set list"); + } + List list = new ArrayList(); + Table table = null; + ResultScanner scanner = null; + try { + table = connection.getTable(tableName); + Scan scan = createScanForBackupSetList(); + scan.setMaxVersions(1); + scanner = table.getScanner(scan); + Result res = null; + while ((res = scanner.next()) != null) { + res.advance(); + list.add(cellKeyToBackupSetName(res.current())); + } + return list; + } finally { + if (scanner != null) { + scanner.close(); + } + if (table != null) { + table.close(); + } + } + } + + /** + * Get backup set description (list of tables) + * @param name set's name + * @return list of tables in a backup set + * @throws IOException + */ + public List describeBackupSet(String name) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set describe: " + name); + } + Table table = null; + try { + table = connection.getTable(tableName); + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) return null; + res.advance(); + String[] tables = cellValueToBackupSet(res.current()); + return toList(tables); + } finally { + if (table != null) { + table.close(); + } + } + } + + private List toList(String[] tables) { + List list = new ArrayList(tables.length); + for (String name : tables) { + list.add(TableName.valueOf(name)); + } + return list; + } + + /** + * Add backup set (list of tables) + * @param name set name + * @param newTables list of tables, comma-separated + * @throws IOException + */ + public void addToBackupSet(String name, String[] newTables) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]"); + } + Table table = null; + String[] union = null; + try { + table = connection.getTable(tableName); + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + union = newTables; + } else { + res.advance(); + String[] tables = cellValueToBackupSet(res.current()); + union = merge(tables, newTables); + } + Put put = createPutForBackupSet(name, union); + table.put(put); + } finally { + if (table != null) { + table.close(); + } + } + } + + private String[] merge(String[] tables, String[] newTables) { + List list = new ArrayList(); + // Add all from tables + for (String t : tables) { + list.add(t); + } + for (String nt : newTables) { + if (list.contains(nt)) continue; + list.add(nt); + } + String[] arr = new String[list.size()]; + list.toArray(arr); + return arr; + } + + /** + * Remove tables from backup set (list of tables) + * @param name set name + * @param toRemove list of tables + * @throws IOException + */ + public void removeFromBackupSet(String name, String[] toRemove) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ") + + "]"); + } + Table table = null; + String[] disjoint = null; + String[] tables = null; + try { + table = connection.getTable(tableName); + Get get = createGetForBackupSet(name); + Result res = table.get(get); + if (res.isEmpty()) { + LOG.warn("Backup set '" + name + "' not found."); + return; + } else { + res.advance(); + tables = cellValueToBackupSet(res.current()); + disjoint = disjoin(tables, toRemove); + } + if (disjoint.length > 0 && disjoint.length != tables.length) { + Put put = createPutForBackupSet(name, disjoint); + table.put(put); + } else if(disjoint.length == tables.length) { + LOG.warn("Backup set '" + name + "' does not contain tables [" + + StringUtils.join(toRemove, " ") + "]"); + } else { // disjoint.length == 0 and tables.length >0 + // Delete backup set + LOG.info("Backup set '"+name+"' is empty. Deleting."); + deleteBackupSet(name); + } + } finally { + if (table != null) { + table.close(); + } + } + } + + private String[] disjoin(String[] tables, String[] toRemove) { + List list = new ArrayList(); + // Add all from tables + for (String t : tables) { + list.add(t); + } + for (String nt : toRemove) { + if (list.contains(nt)) { + list.remove(nt); + } + } + String[] arr = new String[list.size()]; + list.toArray(arr); + return arr; + } + + /** + * Delete backup set + * @param name set's name + * @throws IOException + */ + public void deleteBackupSet(String name) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace(" Backup set delete: " + name); + } + Table table = null; + try { + table = connection.getTable(tableName); + Delete del = createDeleteForBackupSet(name); + table.delete(del); + } finally { + if (table != null) { + table.close(); + } + } + } + + /** + * Get backup system table descriptor + * @return table's descriptor + */ + public static HTableDescriptor getSystemTableDescriptor(Configuration conf) { + + HTableDescriptor tableDesc = new HTableDescriptor(getTableName(conf)); + HColumnDescriptor colSessionsDesc = new HColumnDescriptor(SESSIONS_FAMILY); + colSessionsDesc.setMaxVersions(1); + // Time to keep backup sessions (secs) + Configuration config = HBaseConfiguration.create(); + int ttl = + config.getInt(BackupRestoreConstants.BACKUP_SYSTEM_TTL_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TTL_DEFAULT); + colSessionsDesc.setTimeToLive(ttl); + tableDesc.addFamily(colSessionsDesc); + HColumnDescriptor colMetaDesc = new HColumnDescriptor(META_FAMILY); + tableDesc.addFamily(colMetaDesc); + return tableDesc; + } + + public static TableName getTableName(Configuration conf) { + String name = + conf.get(BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_KEY, + BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_DEFAULT); + return TableName.valueOf(name); + } + + public static String getTableNameAsString(Configuration conf) { + return getTableName(conf).getNameAsString(); + } + + + + + + /** + * Creates Put operation for a given backup info object + * @param context backup info + * @return put operation + * @throws IOException exception + */ + private Put createPutForBackupInfo(BackupInfo context) throws IOException { + Put put = new Put(rowkey(BACKUP_INFO_PREFIX, context.getBackupId())); + put.addColumn(BackupSystemTable.SESSIONS_FAMILY, Bytes.toBytes("context"), + context.toByteArray()); + return put; + } + + /** + * Creates Get operation for a given backup id + * @param backupId backup's ID + * @return get operation + * @throws IOException exception + */ + private Get createGetForBackupInfo(String backupId) throws IOException { + Get get = new Get(rowkey(BACKUP_INFO_PREFIX, backupId)); + get.addFamily(BackupSystemTable.SESSIONS_FAMILY); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Delete operation for a given backup id + * @param backupId backup's ID + * @return delete operation + * @throws IOException exception + */ + private Delete createDeleteForBackupInfo(String backupId) { + Delete del = new Delete(rowkey(BACKUP_INFO_PREFIX, backupId)); + del.addFamily(BackupSystemTable.SESSIONS_FAMILY); + return del; + } + + /** + * Converts Result to BackupInfo + * @param res HBase result + * @return backup info instance + * @throws IOException exception + */ + private BackupInfo resultToBackupInfo(Result res) throws IOException { + res.advance(); + Cell cell = res.current(); + return cellToBackupInfo(cell); + } + + /** + * Creates Get operation to retrieve start code from backup system table + * @return get operation + * @throws IOException exception + */ + private Get createGetForStartCode(String rootPath) throws IOException { + Get get = new Get(rowkey(START_CODE_ROW, rootPath)); + get.addFamily(BackupSystemTable.META_FAMILY); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Put operation to store start code to backup system table + * @return put operation + * @throws IOException exception + */ + private Put createPutForStartCode(String startCode, String rootPath) { + Put put = new Put(rowkey(START_CODE_ROW, rootPath)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("startcode"), + Bytes.toBytes(startCode)); + return put; + } + + /** + * Creates Get to retrieve incremental backup table set from backup system table + * @return get operation + * @throws IOException exception + */ + private Get createGetForIncrBackupTableSet(String backupRoot) throws IOException { + Get get = new Get(rowkey(INCR_BACKUP_SET, backupRoot)); + get.addFamily(BackupSystemTable.META_FAMILY); + get.setMaxVersions(1); + return get; + } + + /** + * Creates Put to store incremental backup table set + * @param tables tables + * @return put operation + */ + private Put createPutForIncrBackupTableSet(Set tables, String backupRoot) { + Put put = new Put(rowkey(INCR_BACKUP_SET, backupRoot)); + for (TableName table : tables) { + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()), + EMPTY_VALUE); + } + return put; + } + + /** + * Creates Delete for incremental backup table set + * @param backupRoot backup root + * @return delete operation + */ + private Delete createDeleteForIncrBackupTableSet(String backupRoot) { + Delete delete = new Delete(rowkey(INCR_BACKUP_SET, backupRoot)); + delete.addFamily(BackupSystemTable.META_FAMILY); + return delete; + } + + /** + * Creates Scan operation to load backup history + * @return scan operation + */ + private Scan createScanForBackupHistory() { + Scan scan = new Scan(); + byte[] startRow = Bytes.toBytes(BACKUP_INFO_PREFIX); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.SESSIONS_FAMILY); + scan.setMaxVersions(1); + return scan; + } + + /** + * Converts cell to backup info instance. + * @param current current cell + * @return backup backup info instance + * @throws IOException exception + */ + private BackupInfo cellToBackupInfo(Cell current) throws IOException { + byte[] data = CellUtil.cloneValue(current); + return BackupInfo.fromByteArray(data); + } + + /** + * Creates Put to write RS last roll log timestamp map + * @param table table + * @param smap map, containing RS:ts + * @return put operation + */ + private Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap, + String backupRoot) { + Put put = new Put(rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot, NULL, table.getNameAsString())); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("log-roll-map"), smap); + return put; + } + + /** + * Creates Scan to load table-> { RS -> ts} map of maps + * @return scan operation + */ + private Scan createScanForReadLogTimestampMap(String backupRoot) { + Scan scan = new Scan(); + byte[] startRow = rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + + return scan; + } + + /** + * Get table name from rowkey + * @param cloneRow rowkey + * @return table name + */ + private String getTableNameForReadLogTimestampMap(byte[] cloneRow) { + String s = Bytes.toString(cloneRow); + int index = s.lastIndexOf(NULL); + return s.substring(index + 1); + } + + /** + * Creates Put to store RS last log result + * @param server server name + * @param timestamp log roll result (timestamp) + * @return put operation + */ + private Put createPutForRegionServerLastLogRollResult(String server, Long timestamp, + String backupRoot) { + Put put = new Put(rowkey(RS_LOG_TS_PREFIX, backupRoot, NULL, server)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("rs-log-ts"), + Bytes.toBytes(timestamp)); + return put; + } + + /** + * Creates Scan operation to load last RS log roll results + * @return scan operation + */ + private Scan createScanForReadRegionServerLastLogRollResult(String backupRoot) { + Scan scan = new Scan(); + byte[] startRow = rowkey(RS_LOG_TS_PREFIX, backupRoot); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + scan.setMaxVersions(1); + + return scan; + } + + /** + * Get server's name from rowkey + * @param row rowkey + * @return server's name + */ + private String getServerNameForReadRegionServerLastLogRollResult(byte[] row) { + String s = Bytes.toString(row); + int index = s.lastIndexOf(NULL); + return s.substring(index + 1); + } + + /** + * Creates put list for list of WAL files + * @param files list of WAL file paths + * @param backupId backup id + * @return put list + * @throws IOException exception + */ + private List createPutsForAddWALFiles(List files, String backupId, + String backupRoot) throws IOException { + + List puts = new ArrayList(); + for (String file : files) { + Put put = new Put(rowkey(WALS_PREFIX, BackupUtils.getUniqueWALFileNamePart(file))); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("backupId"), + Bytes.toBytes(backupId)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("file"), Bytes.toBytes(file)); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("root"), Bytes.toBytes(backupRoot)); + puts.add(put); + } + return puts; + } + + /** + * Creates Scan operation to load WALs + * @param backupRoot path to backup destination + * @return scan operation + */ + private Scan createScanForGetWALs(String backupRoot) { + // TODO: support for backupRoot + Scan scan = new Scan(); + byte[] startRow = Bytes.toBytes(WALS_PREFIX); + 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.META_FAMILY); + return scan; + } + + /** + * Creates Get operation for a given wal file name TODO: support for backup destination + * @param file file + * @return get operation + * @throws IOException exception + */ + private Get createGetForCheckWALFile(String file) throws IOException { + Get get = new Get(rowkey(WALS_PREFIX, BackupUtils.getUniqueWALFileNamePart(file))); + // add backup root column + get.addFamily(BackupSystemTable.META_FAMILY); + return get; + } + + /** + * Creates Scan operation to load backup set list + * @return scan operation + */ + private Scan createScanForBackupSetList() { + Scan scan = new Scan(); + byte[] startRow = Bytes.toBytes(SET_KEY_PREFIX); + byte[] stopRow = Arrays.copyOf(startRow, startRow.length); + stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.addFamily(BackupSystemTable.META_FAMILY); + return scan; + } + + /** + * Creates Get operation to load backup set content + * @return get operation + */ + private Get createGetForBackupSet(String name) { + Get get = new Get(rowkey(SET_KEY_PREFIX, name)); + get.addFamily(BackupSystemTable.META_FAMILY); + return get; + } + + /** + * Creates Delete operation to delete backup set content + * @param name backup set's name + * @return delete operation + */ + private Delete createDeleteForBackupSet(String name) { + Delete del = new Delete(rowkey(SET_KEY_PREFIX, name)); + del.addFamily(BackupSystemTable.META_FAMILY); + return del; + } + + /** + * Creates Put operation to update backup set content + * @param name backup set's name + * @param tables list of tables + * @return put operation + */ + private Put createPutForBackupSet(String name, String[] tables) { + Put put = new Put(rowkey(SET_KEY_PREFIX, name)); + byte[] value = convertToByteArray(tables); + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("tables"), value); + return put; + } + + private byte[] convertToByteArray(String[] tables) { + return StringUtils.join(tables, ",").getBytes(); + } + + /** + * Converts cell to backup set list. + * @param current current cell + * @return backup set as array of table names + * @throws IOException + */ + private String[] cellValueToBackupSet(Cell current) throws IOException { + byte[] data = CellUtil.cloneValue(current); + if (data != null && data.length > 0) { + return Bytes.toString(data).split(","); + } else { + return new String[0]; + } + } + + /** + * Converts cell key to backup set name. + * @param current current cell + * @return backup set name + * @throws IOException + */ + private String cellKeyToBackupSetName(Cell current) throws IOException { + byte[] data = CellUtil.cloneRow(current); + return Bytes.toString(data).substring(SET_KEY_PREFIX.length()); + } + + private byte[] rowkey(String s, String... other) { + StringBuilder sb = new StringBuilder(s); + for (String ss : other) { + sb.append(ss); + } + return sb.toString().getBytes(); + } + + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java new file mode 100644 index 00000000000..77d11844295 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +/** + * Full table backup implementation + * + */ +@InterfaceAudience.Private +public class FullTableBackupClient extends TableBackupClient { + private static final Log LOG = LogFactory.getLog(FullTableBackupClient.class); + + public FullTableBackupClient(final Connection conn, final String backupId, BackupRequest request) + throws IOException { + super(conn, backupId, request); + } + + /** + * Do snapshot copy. + * @param backupInfo backup info + * @throws Exception exception + */ + private void snapshotCopy(BackupInfo backupInfo) throws Exception { + LOG.info("Snapshot copy is starting."); + + // set overall backup phase: snapshot_copy + backupInfo.setPhase(BackupPhase.SNAPSHOTCOPY); + + // call ExportSnapshot to copy files based on hbase snapshot for backup + // ExportSnapshot only support single snapshot export, need loop for multiple tables case + BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf); + + // number of snapshots matches number of tables + float numOfSnapshots = backupInfo.getSnapshotNames().size(); + + LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied."); + + for (TableName table : backupInfo.getTables()) { + // Currently we simply set the sub copy tasks by counting the table snapshot number, we can + // calculate the real files' size for the percentage in the future. + // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots); + int res = 0; + String[] args = new String[4]; + args[0] = "-snapshot"; + args[1] = backupInfo.getSnapshotName(table); + args[2] = "-copy-to"; + args[3] = backupInfo.getTableBackupDir(table); + + LOG.debug("Copy snapshot " + args[1] + " to " + args[3]); + res = copyService.copy(backupInfo, backupManager, conf, BackupType.FULL, args); + // if one snapshot export failed, do not continue for remained snapshots + if (res != 0) { + LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + "."); + + throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3] + + " with reason code " + res); + } + LOG.info("Snapshot copy " + args[1] + " finished."); + } + } + + /** + * Backup request execution + * @throws IOException + */ + @Override + public void execute() throws IOException { + + try (Admin admin = conn.getAdmin();) { + + // Begin BACKUP + beginBackup(backupManager, backupInfo); + String savedStartCode = null; + boolean firstBackup = false; + // do snapshot for full table backup + + savedStartCode = backupManager.readBackupStartCode(); + firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; + if (firstBackup) { + // This is our first backup. Let's put some marker to system table so that we can hold the logs + // while we do the backup. + backupManager.writeBackupStartCode(0L); + } + // We roll log here before we do the snapshot. It is possible there is duplicate data + // in the log that is already in the snapshot. But if we do it after the snapshot, we + // could have data loss. + // A better approach is to do the roll log on each RS in the same global procedure as + // the snapshot. + LOG.info("Execute roll log procedure for full backup ..."); + + Map props = new HashMap(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + 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 = BackupUtils.getWALFilesOlderThan(conf, newTimestamps); + backupManager.recordWALFiles(logFiles); + } + + // SNAPSHOT_TABLES: + backupInfo.setPhase(BackupPhase.SNAPSHOT); + for (TableName tableName : tableList) { + String snapshotName = + "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_" + + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString(); + + admin.snapshot(snapshotName, tableName); + + backupInfo.setSnapshotName(tableName, snapshotName); + } + + // SNAPSHOT_COPY: + // do snapshot copy + LOG.debug("snapshot copy for " + backupId); + snapshotCopy(backupInfo); + // Updates incremental backup table set + backupManager.addIncrementalBackupTableSet(backupInfo.getTables()); + + // BACKUP_COMPLETE: + // set overall backup status: complete. Here we make sure to complete the backup. + // After this checkpoint, even if entering cancel process, will let the backup finished + backupInfo.setState(BackupState.COMPLETE); + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + HashMap> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + Long newStartCode = + BackupUtils.getMinValue(BackupUtils + .getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf); + } catch (Exception e) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", + BackupType.FULL, conf); + throw new IOException(e); + } + + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java new file mode 100644 index 00000000000..0f1453ee90e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -0,0 +1,344 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +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.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; + +/** + * After a full backup was created, the incremental backup will only store the changes made after + * the last full or incremental backup. Creating the backup copies the logfiles in .logs and + * .oldlogs since the last backup timestamp. + */ +@InterfaceAudience.Private +public class IncrementalBackupManager extends BackupManager { + public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class); + + public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException { + super(conn, conf); + } + + /** + * Obtain the list of logs that need to be copied out for this incremental backup. The list is set + * in BackupInfo. + * @param conn the Connection + * @param backupInfo backup info + * @return The new HashMap of RS log timestamps after the log roll for this incremental backup. + * @throws IOException exception + */ + public HashMap getIncrBackupLogFileList(Connection conn, BackupInfo backupInfo) + throws IOException { + List logList; + HashMap newTimestamps; + HashMap previousTimestampMins; + + String savedStartCode = readBackupStartCode(); + + // key: tableName + // value: + HashMap> previousTimestampMap = readLogTimestampMap(); + + previousTimestampMins = BackupUtils.getRSLogTimestampMins(previousTimestampMap); + + if (LOG.isDebugEnabled()) { + LOG.debug("StartCode " + savedStartCode + "for backupID " + backupInfo.getBackupId()); + } + // get all new log files from .logs and .oldlogs after last TS and before new timestamp + if (savedStartCode == null || previousTimestampMins == null + || previousTimestampMins.isEmpty()) { + throw new IOException( + "Cannot read any previous back up timestamps from backup system table. " + + "In order to create an incremental backup, at least one full backup is needed."); + } + + LOG.info("Execute roll log procedure for incremental backup ..."); + HashMap props = new HashMap(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + + try (Admin admin = conn.getAdmin();) { + + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + + } + newTimestamps = readRegionServerLastLogRollResult(); + + logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode); + List logFromSystemTable = + getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo() + .getBackupRootDir()); + addLogsFromBackupSystemToContext(logFromSystemTable); + + logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable); + backupInfo.setIncrBackupFileList(logList); + + return newTimestamps; + } + + private List excludeAlreadyBackedUpWALs(List logList, + List logFromSystemTable) { + + List backupedWALList = toWALList(logFromSystemTable); + logList.removeAll(backupedWALList); + return logList; + } + + private List toWALList(List logFromSystemTable) { + + List list = new ArrayList(logFromSystemTable.size()); + for (WALItem item : logFromSystemTable) { + list.add(item.getWalFile()); + } + return list; + } + + private void addLogsFromBackupSystemToContext(List logFromSystemTable) { + List walFiles = new ArrayList(); + for (WALItem item : logFromSystemTable) { + Path p = new Path(item.getWalFile()); + String walFileName = p.getName(); + String backupId = item.getBackupId(); + String relWALPath = backupId + Path.SEPARATOR + walFileName; + walFiles.add(relWALPath); + } + } + + /** + * For each region server: get all log files newer than the last timestamps, but not newer than + * the newest timestamps. + * @param olderTimestamps timestamp map for each region server of the last backup. + * @param newestTimestamps timestamp map for each region server that the backup should lead to. + * @return list of log files which needs to be added to this backup + * @throws IOException + */ + private List getLogFilesFromBackupSystem(HashMap olderTimestamps, + HashMap newestTimestamps, String backupRoot) throws IOException { + List logFiles = new ArrayList(); + Iterator it = getWALFilesFromBackupSystem(); + while (it.hasNext()) { + WALItem item = it.next(); + String rootDir = item.getBackupRoot(); + if (!rootDir.equals(backupRoot)) { + continue; + } + String walFileName = item.getWalFile(); + String server = BackupUtils.parseHostNameFromLogFile(new Path(walFileName)); + if (server == null) { + continue; + } + Long tss = getTimestamp(walFileName); + Long oldTss = olderTimestamps.get(server); + Long newTss = newestTimestamps.get(server); + if (oldTss == null) { + logFiles.add(item); + continue; + } + if (newTss == null) { + newTss = Long.MAX_VALUE; + } + if (tss > oldTss && tss < newTss) { + logFiles.add(item); + } + } + return logFiles; + } + + private Long getTimestamp(String walFileName) { + int index = walFileName.lastIndexOf(BackupUtils.LOGNAME_SEPARATOR); + return Long.parseLong(walFileName.substring(index + 1)); + } + + /** + * 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(); + + List resultLogFiles = new ArrayList(); + List newestLogs = new ArrayList(); + + /* + * The old region servers and timestamps info we kept in backup system table may be out of sync + * if new region server is added or existing one lost. We'll deal with it here when processing + * the logs. If data in backup system table has more hosts, just ignore it. If the .logs + * directory includes more hosts, the additional hosts will not have old timestamps to compare + * with. We'll just use all the logs in that directory. We always write up-to-date region server + * and timestamp info to backup system table at the end of successful backup. + */ + + FileStatus[] rss; + Path p; + String host; + Long oldTimeStamp; + String currentLogFile; + long currentLogTS; + + // Get the files in .logs. + rss = fs.listStatus(logDir); + for (FileStatus rs : rss) { + p = rs.getPath(); + host = BackupUtils.parseHostNameFromLogFile(p); + if (host == null) { + continue; + } + FileStatus[] logs; + oldTimeStamp = olderTimestamps.get(host); + // It is possible that there is no old timestamp in backup system table for this host if + // this region server is newly added after our last backup. + if (oldTimeStamp == null) { + logs = fs.listStatus(p); + } else { + pathFilter.setLastBackupTS(oldTimeStamp); + logs = fs.listStatus(p, pathFilter); + } + for (FileStatus log : logs) { + LOG.debug("currentLogFile: " + log.getPath().toString()); + if (AbstractFSWALProvider.isMetaFile(log.getPath())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip hbase:meta log file: " + log.getPath().getName()); + } + continue; + } + currentLogFile = log.getPath().toString(); + resultLogFiles.add(currentLogFile); + currentLogTS = BackupUtils.getCreationTime(log.getPath()); + // newestTimestamps is up-to-date with the current list of hosts + // so newestTimestamps.get(host) will not be null. + if (currentLogTS > 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 (AbstractFSWALProvider.isMetaFile(p)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip .meta log file: " + currentLogFile); + } + continue; + } + host = BackupUtils.parseHostFromOldLog(p); + if (host == null) { + continue; + } + currentLogTS = BackupUtils.getCreationTime(p); + oldTimeStamp = olderTimestamps.get(host); + /* + * It is possible that there is no old timestamp in backup system table for this host. At the + * time of our last backup operation, this rs did not exist. The reason can be one of the two: + * 1. The rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after + * our last backup. + */ + if (oldTimeStamp == null) { + if (currentLogTS < Long.parseLong(savedStartCode)) { + // This log file is really old, its region server was before our last backup. + continue; + } else { + resultLogFiles.add(currentLogFile); + } + } else if (currentLogTS > oldTimeStamp) { + resultLogFiles.add(currentLogFile); + } + + // It is possible that a host in .oldlogs is an obsolete region server + // so newestTimestamps.get(host) here can be null. + // Even if these logs belong to a obsolete region server, we still need + // to include they to avoid loss of edits for backup. + Long newTimestamp = newestTimestamps.get(host); + if (newTimestamp != null && currentLogTS > newTimestamp) { + newestLogs.add(currentLogFile); + } + } + // remove newest log per host because they are still in use + resultLogFiles.removeAll(newestLogs); + return resultLogFiles; + } + + static class NewestLogFilter implements PathFilter { + private Long lastBackupTS = 0L; + + public NewestLogFilter() { + } + + protected void setLastBackupTS(Long ts) { + this.lastBackupTS = ts; + } + + @Override + public boolean accept(Path path) { + // skip meta table log -- ts.meta file + if (AbstractFSWALProvider.isMetaFile(path)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip .meta log file: " + path.getName()); + } + return false; + } + long timestamp; + try { + timestamp = BackupUtils.getCreationTime(path); + return timestamp > lastBackupTS; + } catch (Exception e) { + LOG.warn("Cannot read timestamp of log file " + path); + return false; + } + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java new file mode 100644 index 00000000000..395ed6d986d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -0,0 +1,216 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; + +/** + * Incremental backup implementation. + * See the {@link #execute() execute} method. + * + */ +@InterfaceAudience.Private +public class IncrementalTableBackupClient extends TableBackupClient { + private static final Log LOG = LogFactory.getLog(IncrementalTableBackupClient.class); + + public IncrementalTableBackupClient(final Connection conn, final String backupId, + BackupRequest request) throws IOException { + super(conn, backupId, request); + } + + private List filterMissingFiles(List incrBackupFileList) throws IOException { + FileSystem fs = FileSystem.get(conf); + List list = new ArrayList(); + for (String file : incrBackupFileList) { + if (fs.exists(new Path(file))) { + list.add(file); + } else { + LOG.warn("Can't find file: " + file); + } + } + return list; + } + + private List getMissingFiles(List incrBackupFileList) throws IOException { + FileSystem fs = FileSystem.get(conf); + List list = new ArrayList(); + for (String file : incrBackupFileList) { + if (!fs.exists(new Path(file))) { + list.add(file); + } + } + return list; + + } + + /** + * Do incremental copy. + * @param backupInfo backup info + */ + private void incrementalCopy(BackupInfo backupInfo) throws Exception { + + LOG.info("Incremental copy is starting."); + // set overall backup phase: incremental_copy + backupInfo.setPhase(BackupPhase.INCREMENTAL_COPY); + // get incremental backup file list and prepare parms for DistCp + List incrBackupFileList = backupInfo.getIncrBackupFileList(); + // filter missing files out (they have been copied by previous backups) + incrBackupFileList = filterMissingFiles(incrBackupFileList); + String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]); + strArr[strArr.length - 1] = backupInfo.getHLogTargetDir(); + + BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf); + int counter = 0; + int MAX_ITERAIONS = 2; + while (counter++ < MAX_ITERAIONS) { + // We run DistCp maximum 2 times + // If it fails on a second time, we throw Exception + int res = + copyService.copy(backupInfo, backupManager, conf, BackupType.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 " + + StringUtils.join(incrBackupFileList, ",") + " to " + + backupInfo.getHLogTargetDir()); + } + List missingFiles = getMissingFiles(incrBackupFileList); + + if (missingFiles.isEmpty()) { + break; + } else { + // Repeat DistCp, some files have been moved from WALs to oldWALs during previous run + // update backupInfo and strAttr + if (counter == MAX_ITERAIONS) { + String msg = + "DistCp could not finish the following files: " + StringUtils.join(missingFiles, ","); + LOG.error(msg); + throw new IOException(msg); + } + List converted = convertFilesFromWALtoOldWAL(missingFiles); + incrBackupFileList.removeAll(missingFiles); + incrBackupFileList.addAll(converted); + backupInfo.setIncrBackupFileList(incrBackupFileList); + + // Run DistCp only for missing files (which have been moved from WALs to oldWALs + // during previous run) + strArr = converted.toArray(new String[converted.size() + 1]); + strArr[strArr.length - 1] = backupInfo.getHLogTargetDir(); + } + } + + LOG.info("Incremental copy from " + StringUtils.join(incrBackupFileList, ",") + " to " + + backupInfo.getHLogTargetDir() + " finished."); + } + + private List convertFilesFromWALtoOldWAL(List missingFiles) throws IOException { + List list = new ArrayList(); + for (String path : missingFiles) { + if (path.indexOf(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME) < 0) { + LOG.error("Copy incremental log files failed, file is missing : " + path); + throw new IOException("Failed of Hadoop Distributed Copy to " + + backupInfo.getHLogTargetDir() + ", file is missing " + path); + } + list.add(path.replace(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME, Path.SEPARATOR + + HConstants.HREGION_OLDLOGDIR_NAME)); + } + return list; + } + + @Override + public void execute() throws IOException { + + // case PREPARE_INCREMENTAL: + beginBackup(backupManager, backupInfo); + backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); + LOG.debug("For incremental backup, current table set is " + + backupManager.getIncrementalBackupTableSet()); + try { + newTimestamps = + ((IncrementalBackupManager) backupManager).getIncrBackupLogFileList(conn, backupInfo); + } catch (Exception e) { + // fail the overall backup and return + failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", + BackupType.INCREMENTAL, conf); + } + + // case INCREMENTAL_COPY: + try { + // copy out the table and region info files for each table + BackupUtils.copyTableRegionInfo(conn, backupInfo, conf); + incrementalCopy(backupInfo); + // Save list of WAL files copied + backupManager.recordWALFiles(backupInfo.getIncrBackupFileList()); + } catch (Exception e) { + String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId; + // fail the overall backup and return + failBackup(conn, backupInfo, backupManager, e, msg, BackupType.INCREMENTAL, conf); + } + // case INCR_BACKUP_COMPLETE: + // set overall backup status: complete. Here we make sure to complete the backup. + // After this checkpoint, even if entering cancel process, will let the backup finished + try { + backupInfo.setState(BackupState.COMPLETE); + // Set the previousTimestampMap which is before this current log roll to the manifest. + HashMap> previousTimestampMap = + backupManager.readLogTimestampMap(); + backupInfo.setIncrTimestampMap(previousTimestampMap); + + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + HashMap> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + Long newStartCode = + BackupUtils.getMinValue(BackupUtils + .getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + // backup complete + completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf); + + } catch (IOException e) { + failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", + BackupType.INCREMENTAL, conf); + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java new file mode 100644 index 00000000000..f418305702a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java @@ -0,0 +1,237 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.TreeSet; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.backup.util.RestoreTool; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; + +/** + * Restore table implementation + * + */ +@InterfaceAudience.Private +public class RestoreTablesClient { + private static final Log LOG = LogFactory.getLog(RestoreTablesClient.class); + + private Configuration conf; + private Connection conn; + private String backupId; + private TableName[] sTableArray; + private TableName[] tTableArray; + private String targetRootDir; + private boolean isOverwrite; + + public RestoreTablesClient(Connection conn, RestoreRequest request) throws IOException { + this.targetRootDir = request.getBackupRootDir(); + this.backupId = request.getBackupId(); + this.sTableArray = request.getFromTables(); + this.tTableArray = request.getToTables(); + if (tTableArray == null || tTableArray.length == 0) { + this.tTableArray = sTableArray; + } + this.isOverwrite = request.isOverwrite(); + this.conn = conn; + this.conf = conn.getConfiguration(); + + } + + /** + * Validate target tables + * @param conn connection + * @param mgr table state manager + * @param tTableArray: target tables + * @param isOverwrite overwrite existing table + * @throws IOException exception + */ + private void checkTargetTables(TableName[] tTableArray, boolean isOverwrite) throws IOException { + ArrayList existTableList = new ArrayList<>(); + ArrayList disabledTableList = new ArrayList<>(); + + // check if the tables already exist + try (Admin admin = conn.getAdmin();) { + for (TableName tableName : tTableArray) { + if (admin.tableExists(tableName)) { + existTableList.add(tableName); + if (admin.isTableDisabled(tableName)) { + disabledTableList.add(tableName); + } + } else { + LOG.info("HBase table " + tableName + + " does not exist. It will be created during restore process"); + } + } + } + + if (existTableList.size() > 0) { + if (!isOverwrite) { + LOG.error("Existing table (" + + existTableList + + ") found in the restore target, please add " + + "\"-overwrite\" option in the command if you mean" + + " to restore to these existing tables"); + 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 handle each backupImage in array + * @param svc: master services + * @param images: array BackupImage + * @param sTable: table to be restored + * @param tTable: table to be restored to + * @param truncateIfExists: truncate table + * @throws IOException exception + */ + + private void restoreImages(BackupImage[] images, TableName sTable, TableName tTable, + boolean truncateIfExists) throws IOException { + + // First image MUST be image of a FULL backup + BackupImage image = images[0]; + String rootDir = image.getRootDir(); + String backupId = image.getBackupId(); + Path backupRoot = new Path(rootDir); + RestoreTool restoreTool = new RestoreTool(conf, backupRoot, backupId); + Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId); + String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId(); + // We need hFS only for full restore (see the code) + BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId); + if (manifest.getType() == BackupType.FULL) { + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full" + " backup image " + + tableBackupPath.toString()); + restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists, + lastIncrBackupId); + } else { // incremental Backup + throw new IOException("Unexpected backup type " + image.getType()); + } + + if (images.length == 1) { + // full backup restore done + return; + } + + List dirList = new ArrayList(); + // add full backup path + // full backup path comes first + for (int i = 1; i < images.length; i++) { + BackupImage im = images[i]; + String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId()); + dirList.add(new Path(logBackupDir)); + } + + String dirs = StringUtils.join(dirList, ","); + LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from log dirs: " + dirs); + Path[] paths = new Path[dirList.size()]; + dirList.toArray(paths); + restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable }, + new TableName[] { tTable }, lastIncrBackupId); + LOG.info(sTable + " has been successfully restored to " + tTable); + + } + + /** + * Restore operation. Stage 2: resolved Backup Image dependency + * @param backupManifestMap : tableName, Manifest + * @param sTableArray The array of tables to be restored + * @param tTableArray The array of mapping tables to restore to + * @return set of BackupImages restored + * @throws IOException exception + */ + private void restore(HashMap backupManifestMap, + TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException { + TreeSet restoreImageSet = new TreeSet(); + boolean truncateIfExists = isOverwrite; + try { + for (int i = 0; i < sTableArray.length; i++) { + TableName table = sTableArray[i]; + BackupManifest manifest = backupManifestMap.get(table); + // Get the image list of this backup for restore in time order from old + // to new. + List list = new ArrayList(); + list.add(manifest.getBackupImage()); + TreeSet set = new TreeSet(list); + List depList = manifest.getDependentListByTable(table); + set.addAll(depList); + BackupImage[] arr = new BackupImage[set.size()]; + set.toArray(arr); + restoreImages(arr, table, tTableArray[i], truncateIfExists); + restoreImageSet.addAll(list); + if (restoreImageSet != null && !restoreImageSet.isEmpty()) { + LOG.info("Restore includes the following image(s):"); + for (BackupImage image : restoreImageSet) { + LOG.info("Backup: " + + image.getBackupId() + + " " + + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), + table)); + } + } + } + } catch (Exception e) { + LOG.error("Failed", e); + throw new IOException(e); + } + LOG.debug("restoreStage finished"); + } + + public void execute() throws IOException { + + // case VALIDATION: + // check the target tables + checkTargetTables(tTableArray, isOverwrite); + // case RESTORE_IMAGES: + HashMap backupManifestMap = new HashMap<>(); + // check and load backup image manifest for the tables + Path rootPath = new Path(targetRootDir); + HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath, + backupId); + restore(backupManifestMap, sTableArray, tTableArray, isOverwrite); + } + + + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java new file mode 100644 index 00000000000..42a80766ab4 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java @@ -0,0 +1,387 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import 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.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupRequest; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; + +/** + * Base class for backup operation. Concrete implementation for + * full and incremental backup are delegated to corresponding sub-classes: + * {@link FullTableBackupClient} and {@link IncrementalTableBackupClient} + * + */ +@InterfaceAudience.Private +public abstract class TableBackupClient { + private static final Log LOG = LogFactory.getLog(TableBackupClient.class); + + protected Configuration conf; + protected Connection conn; + protected String backupId; + protected List tableList; + protected HashMap newTimestamps = null; + + protected BackupManager backupManager; + protected BackupInfo backupInfo; + + public TableBackupClient(final Connection conn, final String backupId, BackupRequest request) + throws IOException { + if (request.getBackupType() == BackupType.FULL) { + backupManager = new BackupManager(conn, conn.getConfiguration()); + } else { + backupManager = new IncrementalBackupManager(conn, conn.getConfiguration()); + } + this.backupId = backupId; + this.tableList = request.getTableList(); + this.conn = conn; + this.conf = conn.getConfiguration(); + backupInfo = + backupManager.createBackupInfo(backupId, request.getBackupType(), tableList, + request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth()); + if (tableList == null || tableList.isEmpty()) { + this.tableList = new ArrayList<>(backupInfo.getTables()); + } + } + + /** + * Begin the overall backup. + * @param backupInfo backup info + * @throws IOException exception + */ + protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) + throws IOException { + backupManager.setBackupInfo(backupInfo); + // set the start timestamp of the overall backup + long startTs = EnvironmentEdgeManager.currentTime(); + backupInfo.setStartTs(startTs); + // set overall backup status: ongoing + backupInfo.setState(BackupState.RUNNING); + backupInfo.setPhase(BackupPhase.REQUEST); + LOG.info("Backup " + backupInfo.getBackupId() + " started at " + startTs + "."); + + backupManager.updateBackupInfo(backupInfo); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup session " + backupInfo.getBackupId() + " has been started."); + } + } + + private String getMessage(Exception e) { + String msg = e.getMessage(); + if (msg == null || msg.equals("")) { + msg = e.getClass().getName(); + } + return msg; + } + + /** + * Delete HBase snapshot for backup. + * @param backupInfo backup info + * @throws Exception exception + */ + private void deleteSnapshot(final Connection conn, BackupInfo backupInfo, Configuration conf) + throws IOException { + LOG.debug("Trying to delete snapshot for full backup."); + for (String snapshotName : backupInfo.getSnapshotNames()) { + if (snapshotName == null) { + continue; + } + LOG.debug("Trying to delete snapshot: " + snapshotName); + + try (Admin admin = conn.getAdmin();) { + admin.deleteSnapshot(snapshotName); + } catch (IOException ioe) { + LOG.debug("when deleting snapshot " + snapshotName, ioe); + } + LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupInfo.getBackupId() + + " succeeded."); + } + } + + /** + * Clean up directories with prefix "exportSnapshot-", which are generated when exporting + * snapshots. + * @throws IOException exception + */ + private void cleanupExportSnapshotLog(Configuration conf) throws IOException { + FileSystem fs = FSUtils.getCurrentFileSystem(conf); + Path stagingDir = + new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory() + .toString())); + FileStatus[] files = FSUtils.listStatus(fs, stagingDir); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("exportSnapshot-")) { + LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName()); + if (FSUtils.delete(fs, file.getPath(), true) == false) { + LOG.warn("Can not delete " + file.getPath()); + } + } + } + } + + /** + * Clean up the uncompleted data at target directory if the ongoing backup has already entered + * the copy phase. + */ + private void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) { + try { + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + LOG.debug("Trying to cleanup up target dir. Current backup phase: " + + backupInfo.getPhase()); + if (backupInfo.getPhase().equals(BackupPhase.SNAPSHOTCOPY) + || backupInfo.getPhase().equals(BackupPhase.INCREMENTAL_COPY) + || backupInfo.getPhase().equals(BackupPhase.STORE_MANIFEST)) { + FileSystem outputFs = + FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + // now treat one backup as a transaction, clean up data that has been partially copied at + // table level + for (TableName table : backupInfo.getTables()) { + Path targetDirPath = + new Path(HBackupFileSystem.getTableBackupDir(backupInfo.getBackupRootDir(), + backupInfo.getBackupId(), table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.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 " + backupInfo.getBackupId() + " at " + + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Fail the overall backup. + * @param backupInfo backup info + * @param e exception + * @throws Exception exception + */ + protected void failBackup(Connection conn, BackupInfo backupInfo, BackupManager backupManager, + Exception e, String msg, BackupType type, Configuration conf) throws IOException { + LOG.error(msg + getMessage(e), e); + // If this is a cancel exception, then we've already cleaned. + + // set the failure timestamp of the overall backup + backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); + + // set failure message + backupInfo.setFailedMsg(e.getMessage()); + + // set overall backup status: failed + backupInfo.setState(BackupState.FAILED); + + // compose the backup failed data + String backupFailedData = + "BackupId=" + backupInfo.getBackupId() + ",startts=" + backupInfo.getStartTs() + + ",failedts=" + backupInfo.getCompleteTs() + ",failedphase=" + backupInfo.getPhase() + + ",failedmessage=" + backupInfo.getFailedMsg(); + LOG.error(backupFailedData); + + backupManager.updateBackupInfo(backupInfo); + + // if full backup, then delete HBase snapshots if there already are snapshots taken + // and also clean up export snapshot log files if exist + if (type == BackupType.FULL) { + deleteSnapshot(conn, backupInfo, conf); + cleanupExportSnapshotLog(conf); + } + + // clean up the uncompleted data at target directory if the ongoing backup has already entered + // the copy phase + // For incremental backup, DistCp logs will be cleaned with the targetDir. + cleanupTargetDir(backupInfo, conf); + LOG.info("Backup " + backupInfo.getBackupId() + " failed."); + } + + /** + * Add manifest for the current backup. The manifest is stored within the table backup directory. + * @param backupInfo The current backup info + * @throws IOException exception + * @throws BackupException exception + */ + private void addManifest(BackupInfo backupInfo, BackupManager backupManager, BackupType type, + Configuration conf) throws IOException, BackupException { + // set the overall backup phase : store manifest + backupInfo.setPhase(BackupPhase.STORE_MANIFEST); + + BackupManifest manifest; + + // Since we have each table's backup in its own directory structure, + // we'll store its manifest with the table directory. + for (TableName table : backupInfo.getTables()) { + manifest = new BackupManifest(backupInfo, table); + ArrayList ancestors = backupManager.getAncestors(backupInfo, table); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + + if (type == BackupType.INCREMENTAL) { + // We'll store the log timestamps for this table only in its manifest. + HashMap> tableTimestampMap = + new HashMap>(); + tableTimestampMap.put(table, backupInfo.getIncrTimestampMap().get(table)); + manifest.setIncrTimestampMap(tableTimestampMap); + ArrayList ancestorss = backupManager.getAncestors(backupInfo); + for (BackupImage image : ancestorss) { + manifest.addDependentImage(image); + } + } + manifest.store(conf); + } + + // For incremental backup, we store a overall manifest in + // /WALs/ + // This is used when created the next incremental backup + if (type == BackupType.INCREMENTAL) { + manifest = new BackupManifest(backupInfo); + // set the table region server start and end timestamps for incremental backup + manifest.setIncrTimestampMap(backupInfo.getIncrTimestampMap()); + ArrayList ancestors = backupManager.getAncestors(backupInfo); + for (BackupImage image : ancestors) { + manifest.addDependentImage(image); + } + manifest.store(conf); + } + } + + /** + * Get backup request meta data dir as string. + * @param backupInfo backup info + * @return meta data dir + */ + private String obtainBackupMetaDataStr(BackupInfo backupInfo) { + StringBuffer sb = new StringBuffer(); + sb.append("type=" + backupInfo.getType() + ",tablelist="); + for (TableName table : backupInfo.getTables()) { + sb.append(table + ";"); + } + if (sb.lastIndexOf(";") > 0) { + sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1); + } + sb.append(",targetRootDir=" + backupInfo.getBackupRootDir()); + + return sb.toString(); + } + + /** + * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying + * hlogs. + * @throws IOException exception + */ + private void cleanupDistCpLog(BackupInfo backupInfo, Configuration conf) throws IOException { + Path rootPath = new Path(backupInfo.getHLogTargetDir()).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = FSUtils.listStatus(fs, rootPath); + if (files == null) { + return; + } + for (FileStatus file : files) { + if (file.getPath().getName().startsWith("_distcp_logs")) { + LOG.debug("Delete log files of DistCp: " + file.getPath().getName()); + FSUtils.delete(fs, file.getPath(), true); + } + } + } + + /** + * Complete the overall backup. + * @param backupInfo backup info + * @throws Exception exception + */ + protected void completeBackup(final Connection conn, BackupInfo backupInfo, + BackupManager backupManager, BackupType type, Configuration conf) throws IOException { + // set the complete timestamp of the overall backup + backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime()); + // set overall backup status: complete + backupInfo.setState(BackupState.COMPLETE); + backupInfo.setProgress(100); + // add and store the manifest for the backup + addManifest(backupInfo, backupManager, type, conf); + + // after major steps done and manifest persisted, do convert if needed for incremental backup + /* in-fly convert code here, provided by future jira */ + LOG.debug("in-fly convert code here, provided by future jira"); + + // compose the backup complete data + String backupCompleteData = + obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs() + + ",completets=" + backupInfo.getCompleteTs() + ",bytescopied=" + + backupInfo.getTotalBytesCopied(); + if (LOG.isDebugEnabled()) { + LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData); + } + backupManager.updateBackupInfo(backupInfo); + + // when full backup is done: + // - delete HBase snapshot + // - clean up directories with prefix "exportSnapshot-", which are generated when exporting + // snapshots + if (type == BackupType.FULL) { + deleteSnapshot(conn, backupInfo, conf); + cleanupExportSnapshotLog(conf); + } else if (type == BackupType.INCREMENTAL) { + cleanupDistCpLog(backupInfo, conf); + } + LOG.info("Backup " + backupInfo.getBackupId() + " completed."); + } + + /** + * Backup request execution + * @throws IOException + */ + public abstract void execute() throws IOException; + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java new file mode 100644 index 00000000000..564172073fd --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java @@ -0,0 +1,181 @@ +/** + * 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.conf.Configured; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.HFileInputFormat; +import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2; +import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +/** + * A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles + * for later bulk importing. + */ +@InterfaceAudience.Private +public class HFileSplitterJob extends Configured implements Tool { + private static final Log LOG = LogFactory.getLog(HFileSplitterJob.class); + final static String NAME = "HFileSplitterJob"; + public final static String BULK_OUTPUT_CONF_KEY = "hfile.bulk.output"; + public final static String TABLES_KEY = "hfile.input.tables"; + public final static String TABLE_MAP_KEY = "hfile.input.tablesmap"; + private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; + + public HFileSplitterJob() { + } + + protected HFileSplitterJob(final Configuration c) { + super(c); + } + + /** + * A mapper that just writes out cells. This one can be used together with + * {@link KeyValueSortReducer} + */ + static class HFileCellMapper extends + Mapper { + + @Override + public void map(NullWritable key, KeyValue value, Context context) throws IOException, + InterruptedException { + // Convert value to KeyValue if subclass + if (!value.getClass().equals(KeyValue.class)) { + value = + new KeyValue(value.getRowArray(), value.getRowOffset(), value.getRowLength(), + value.getFamilyArray(), value.getFamilyOffset(), value.getFamilyLength(), + value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(), + value.getTimestamp(), Type.codeToType(value.getTypeByte()), value.getValueArray(), + value.getValueOffset(), value.getValueLength()); + } + context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), value); + } + + @Override + public void setup(Context context) throws IOException { + // do nothing + } + } + + /** + * Sets up the actual job. + * @param args The command line parameters. + * @return The newly created job. + * @throws IOException When setting up the job fails. + */ + public Job createSubmittableJob(String[] args) throws IOException { + Configuration conf = getConf(); + String inputDirs = args[0]; + String tabName = args[1]; + conf.setStrings(TABLES_KEY, tabName); + Job job = + Job.getInstance(conf, + conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime())); + job.setJarByClass(HFileSplitterJob.class); + FileInputFormat.addInputPaths(job, inputDirs); + job.setInputFormatClass(HFileInputFormat.class); + job.setMapOutputKeyClass(ImmutableBytesWritable.class); + String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); + if (hfileOutPath != null) { + LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs); + TableName tableName = TableName.valueOf(tabName); + job.setMapperClass(HFileCellMapper.class); + job.setReducerClass(KeyValueSortReducer.class); + Path outputDir = new Path(hfileOutPath); + FileOutputFormat.setOutputPath(job, outputDir); + job.setMapOutputValueClass(KeyValue.class); + try (Connection conn = ConnectionFactory.createConnection(conf); + Table table = conn.getTable(tableName); + RegionLocator regionLocator = conn.getRegionLocator(tableName)) { + HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator); + } + LOG.debug("success configuring load incremental job"); + + TableMapReduceUtil.addDependencyJars(job.getConfiguration(), + com.google.common.base.Preconditions.class); + } else { + throw new IOException("No bulk output directory specified"); + } + return job; + } + + /** + * Print usage + * @param errorMsg Error message. Can be null. + */ + private void usage(final String errorMsg) { + if (errorMsg != null && errorMsg.length() > 0) { + System.err.println("ERROR: " + errorMsg); + } + System.err.println("Usage: " + NAME + " [options] "); + System.err.println("Read all HFile's for
and split them to
region boundaries."); + System.err.println("
table to load.\n"); + System.err.println("To generate HFiles for a bulk data load, pass the option:"); + System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output"); + System.err.println("Other options:"); + System.err.println(" -D " + JOB_NAME_CONF_KEY + + "=jobName - use the specified mapreduce job name for the HFile splitter"); + System.err.println("For performance also consider the following options:\n" + + " -Dmapreduce.map.speculative=false\n" + " -Dmapreduce.reduce.speculative=false"); + } + + /** + * Main entry point. + * @param args The command line parameters. + * @throws Exception When running the job fails. + */ + public static void main(String[] args) throws Exception { + int ret = ToolRunner.run(new HFileSplitterJob(HBaseConfiguration.create()), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length < 2) { + usage("Wrong number of arguments: " + args.length); + System.exit(-1); + } + Job job = createSubmittableJob(args); + int result = job.waitForCompletion(true) ? 0 : 1; + return result; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java new file mode 100644 index 00000000000..016d1a45e32 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java @@ -0,0 +1,344 @@ +/** + * 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.TableName; +import org.apache.hadoop.hbase.backup.BackupCopyJob; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.snapshot.ExportSnapshot; +import org.apache.hadoop.mapreduce.Cluster; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.tools.DistCp; +import org.apache.hadoop.tools.DistCpConstants; +import org.apache.hadoop.tools.DistCpOptions; +import org.apache.zookeeper.KeeperException.NoNodeException; + +/** + * Map-Reduce implementation of {@link BackupCopyJob}. Basically, there are 2 types of copy + * operation: one is copying from snapshot, which bases on extending ExportSnapshot's function, the + * other is copying for incremental log files, which bases on extending DistCp's function. + */ +@InterfaceAudience.Private +public class MapReduceBackupCopyJob implements BackupCopyJob { + private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyJob.class); + + private Configuration conf; + + // Accumulated progress within the whole backup process for the copy operation + private float progressDone = 0.1f; + private long bytesCopied = 0; + private static float INIT_PROGRESS = 0.1f; + + // The percentage of the current copy task within the whole task if multiple time copies are + // needed. The default value is 100%, which means only 1 copy task for the whole. + private float subTaskPercntgInWholeTask = 1f; + + public MapReduceBackupCopyJob() { + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + /** + * Get the current copy task percentage within the whole task if multiple copies are needed. + * @return the current copy task percentage + */ + public float getSubTaskPercntgInWholeTask() { + return subTaskPercntgInWholeTask; + } + + /** + * Set the current copy task percentage within the whole task if multiple copies are needed. Must + * be called before calling + * {@link #copy(BackupInfo, BackupManager, Configuration, BackupType, String[])} + * @param subTaskPercntgInWholeTask The percentage of the copy subtask + */ + public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) { + this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask; + } + + static class SnapshotCopy extends ExportSnapshot { + private BackupInfo backupInfo; + private TableName table; + + public SnapshotCopy(BackupInfo backupInfo, TableName table) { + super(); + this.backupInfo = backupInfo; + this.table = table; + } + + public TableName getTable() { + return this.table; + } + + public BackupInfo getBackupInfo() { + return this.backupInfo; + } + } + + /** + * Update the ongoing backup with new progress. + * @param backupInfo backup info + * @param newProgress progress + * @param bytesCopied bytes copied + * @throws NoNodeException exception + */ + static void updateProgress(BackupInfo backupInfo, BackupManager backupManager, + int newProgress, long bytesCopied) throws IOException { + // compose the new backup progress data, using fake number for now + String backupProgressData = newProgress + "%"; + + backupInfo.setProgress(newProgress); + backupManager.updateBackupInfo(backupInfo); + LOG.debug("Backup progress data \"" + backupProgressData + + "\" has been updated to backup system table for " + backupInfo.getBackupId()); + } + + /** + * Extends DistCp for progress updating to backup system table + * during backup. Using DistCpV2 (MAPREDUCE-2765). + * Simply extend it and override execute() method to get the + * Job reference for progress updating. + * Only the argument "src1, [src2, [...]] dst" is supported, + * no more DistCp options. + */ + class BackupDistCp extends DistCp { + + private BackupInfo backupInfo; + private BackupManager backupManager; + + public BackupDistCp(Configuration conf, DistCpOptions options, BackupInfo backupInfo, + BackupManager backupManager) throws Exception { + super(conf, options); + this.backupInfo = backupInfo; + this.backupManager = backupManager; + } + + @Override + public Job execute() throws Exception { + + // reflection preparation for private methods and fields + Class classDistCp = org.apache.hadoop.tools.DistCp.class; + Method 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; + + 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(super.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 += + BackupUtils.getFilesLength(aSrc.getFileSystem(super.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 = + MapReduceBackupCopyJob.this.getConf().getInt("hbase.backup.progressreport.frequency", + 500); + float lastProgress = progressDone; + while (!job.isComplete()) { + float newProgress = + progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS); + + if (newProgress > lastProgress) { + + BigDecimal progressData = + new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP); + String newProgressStr = progressData + "%"; + LOG.info("Progress: " + newProgressStr); + updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied); + LOG.debug("Backup progress data updated to backup system table: \"Progress: " + + newProgressStr + ".\""); + lastProgress = newProgress; + } + Thread.sleep(progressReportFreq); + } + // update the progress data after copy job complete + float newProgress = + progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS); + BigDecimal progressData = + new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP); + + String newProgressStr = progressData + "%"; + LOG.info("Progress: " + newProgressStr + " subTask: " + subTaskPercntgInWholeTask + + " mapProgress: " + job.mapProgress()); + + // accumulate the overall backup progress + progressDone = newProgress; + bytesCopied += totalSrcLgth; + + updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied); + LOG.debug("Backup progress data updated to backup system table: \"Progress: " + + newProgressStr + " - " + bytesCopied + " bytes copied.\""); + } catch (Throwable t) { + LOG.error("distcp " + job == null ? "" : job.getJobID() + " encountered error", t); + throw t; + } 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 + " completed: " + job.isComplete() + " " + + job.isSuccessful()); + Counters ctrs = job.getCounters(); + LOG.debug(ctrs); + if (job.isComplete() && !job.isSuccessful()) { + throw new Exception("DistCp job-id: " + jobID + " failed"); + } + + return job; + } + + } + + /** + * Do backup copy based on different types. + * @param context The backup info + * @param conf The hadoop configuration + * @param copyType The backup copy type + * @param options Options for customized ExportSnapshot or DistCp + * @throws Exception exception + */ + @Override + public int copy(BackupInfo context, BackupManager backupManager, Configuration conf, + BackupType copyType, String[] options) throws IOException { + int res = 0; + + try { + if (copyType == BackupType.FULL) { + SnapshotCopy snapshotCp = new SnapshotCopy(context, context.getTableBySnapshot(options[1])); + LOG.debug("Doing SNAPSHOT_COPY"); + // Make a new instance of conf to be used by the snapshot copy class. + snapshotCp.setConf(new Configuration(conf)); + res = snapshotCp.run(options); + + } else if (copyType == BackupType.INCREMENTAL) { + LOG.debug("Doing COPY_TYPE_DISTCP"); + setSubTaskPercntgInWholeTask(1f); + + BackupDistCp distcp = + new BackupDistCp(new Configuration(conf), null, context, backupManager); + // Handle a special case where the source file is a single file. + // In this case, distcp will not create the target dir. It just take the + // target as a file name and copy source file to the target (as a file name). + // We need to create the target dir before run distcp. + LOG.debug("DistCp options: " + Arrays.toString(options)); + Path dest = new Path(options[options.length - 1]); + 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); + } + } + + @Override + public void cancel(String jobId) throws IOException { + JobID id = JobID.forName(jobId); + Cluster cluster = new Cluster(this.getConf()); + try { + Job job = cluster.getJob(id); + if (job == null) { + LOG.error("No job found for " + id); + // should we throw exception + return; + } + if (job.isComplete() || job.isRetired()) { + return; + } + + job.killJob(); + LOG.debug("Killed copy job " + id); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java new file mode 100644 index 00000000000..ffb61ecf25b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java @@ -0,0 +1,182 @@ +/** + * 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.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.RestoreJob; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.Tool; + +/** + * MapReduce implementation of {@link RestoreJob} + * + * For full backup restore, it runs {@link HFileSplitterJob} job and creates + * HFiles which are aligned with a region boundaries of a table being + * restored, for incremental backup restore it runs {@link WALPlayer} in + * bulk load mode (creates HFiles from WAL edits). + * + * The resulting HFiles then are loaded using HBase bulk load tool + * {@link LoadIncrementalHFiles} + */ +@InterfaceAudience.Private +public class MapReduceRestoreJob implements RestoreJob { + public static final Log LOG = LogFactory.getLog(MapReduceRestoreJob.class); + + private Tool player; + private Configuration conf; + + public MapReduceRestoreJob() { + } + + @Override + public void run(Path[] dirPaths, TableName[] tableNames, TableName[] newTableNames, + boolean fullBackupRestore) throws IOException { + + String bulkOutputConfKey; + + if (fullBackupRestore) { + player = new HFileSplitterJob(); + bulkOutputConfKey = HFileSplitterJob.BULK_OUTPUT_CONF_KEY; + } else { + player = new WALPlayer(); + bulkOutputConfKey = WALPlayer.BULK_OUTPUT_CONF_KEY; + } + // Player reads all files in arbitrary directory structure and creates + // a Map task for each file + String dirs = StringUtils.join(dirPaths, ","); + + if (LOG.isDebugEnabled()) { + LOG.debug("Restore " + (fullBackupRestore ? "full" : "incremental") + + " backup from directory " + dirs + " from hbase tables " + + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND) + + " to tables " + + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)); + } + + for (int i = 0; i < tableNames.length; i++) { + + LOG.info("Restore " + tableNames[i] + " into " + newTableNames[i]); + + Path bulkOutputPath = getBulkOutputDir(getFileNameCompatibleString(newTableNames[i])); + Configuration conf = getConf(); + conf.set(bulkOutputConfKey, bulkOutputPath.toString()); + String[] playerArgs = { dirs, tableNames[i].getNameAsString() }; + + int result = 0; + int loaderResult = 0; + try { + + player.setConf(getConf()); + result = player.run(playerArgs); + if (succeeded(result)) { + // do bulk load + LoadIncrementalHFiles loader = createLoader(); + if (LOG.isDebugEnabled()) { + LOG.debug("Restoring HFiles from directory " + bulkOutputPath); + } + String[] args = { bulkOutputPath.toString(), newTableNames[i].getNameAsString() }; + loaderResult = loader.run(args); + + if (failed(loaderResult)) { + throw new IOException("Can not restore from backup directory " + dirs + + " (check Hadoop and HBase logs). Bulk loader return code =" + loaderResult); + } + } else { + throw new IOException("Can not restore from backup directory " + dirs + + " (check Hadoop/MR and HBase logs). Player return code =" + result); + } + LOG.debug("Restore Job finished:" + result); + } catch (Exception e) { + throw new IOException("Can not restore from backup directory " + dirs + + " (check Hadoop and HBase logs) ", e); + } + + } + } + + private String getFileNameCompatibleString(TableName table) { + return table.getNamespaceAsString() + "-" + table.getQualifierAsString(); + } + + private boolean failed(int result) { + return result != 0; + } + + private boolean succeeded(int result) { + return result == 0; + } + + private LoadIncrementalHFiles createLoader() throws IOException { + // set configuration for restore: + // LoadIncrementalHFile needs more time + // hbase.rpc.timeout 600000 + // calculates + Integer milliSecInHour = 3600000; + Configuration conf = new Configuration(getConf()); + conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, milliSecInHour); + + // By default, it is 32 and loader will fail if # of files in any region exceed this + // limit. Bad for snapshot restore. + conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE); + conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes"); + LoadIncrementalHFiles loader = null; + try { + loader = new LoadIncrementalHFiles(conf); + } catch (Exception e) { + throw new IOException(e); + } + return loader; + } + + private Path getBulkOutputDir(String tableName) throws IOException { + Configuration conf = getConf(); + FileSystem fs = FileSystem.get(conf); + String tmp = + conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, + HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY); + Path path = + new Path(tmp + Path.SEPARATOR + "bulk_output-" + tableName + "-" + + EnvironmentEdgeManager.currentTime()); + fs.deleteOnExit(path); + return path; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = 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..b5b887ce8b3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java @@ -0,0 +1,142 @@ +/** + * + * 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.Map; + +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.TableNotFoundException; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MasterServices; +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. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class BackupLogCleaner extends BaseLogCleanerDelegate { + private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class); + + private boolean stopped = false; + private Connection conn; + + public BackupLogCleaner() { + } + + @Override + public void init(Map params) { + if (params != null && params.containsKey(HMaster.MASTER)) { + MasterServices master = (MasterServices) params.get(HMaster.MASTER); + conn = master.getConnection(); + if (getConf() == null) { + super.setConf(conn.getConfiguration()); + } + } + if (conn == null) { + try { + conn = ConnectionFactory.createConnection(getConf()); + } catch (IOException ioe) { + throw new RuntimeException("Failed to create connection", ioe); + } + } + } + + @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 || !BackupManager.isBackupEnabled(getConf())) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return files; + } + + List list = new ArrayList(); + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + // If we do not have recorded backup sessions + try { + if (!table.hasBackupSessions()) { + LOG.trace("BackupLogCleaner has no backup sessions"); + return files; + } + } catch (TableNotFoundException tnfe) { + LOG.warn("backup system table is not available" + tnfe.getMessage()); + return files; + } + + for (FileStatus file : files) { + String wal = file.getPath().toString(); + boolean logInSystemTable = table.isWALFileDeletable(wal); + if (LOG.isDebugEnabled()) { + if (logInSystemTable) { + LOG.debug("Found log file in backup system table, deleting: " + wal); + list.add(file); + } else { + LOG.debug("Didn't find this log in backup system table, keeping: " + wal); + } + } + } + return list; + } catch (IOException e) { + LOG.error("Failed to get backup system table 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(BackupRestoreConstants.BACKUP_ENABLE_KEY, + BackupRestoreConstants.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..47e428c4c4c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java @@ -0,0 +1,155 @@ +/** + * 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.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +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.procedure.RegionServerProcedureManager; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; +import org.apache.zookeeper.KeeperException; + +/** + * Master procedure manager for coordinated cluster-wide WAL roll operation, which is run during + * backup operation, see {@link MasterProcedureManager} and and {@link RegionServerProcedureManager} + */ +@InterfaceAudience.Private +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 { + if (!isBackupEnabled()) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + this.done = false; + // start the process on the RS + ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance()); + List serverNames = master.getServerManager().getOnlineServersList(); + List servers = new ArrayList(); + for (ServerName sn : serverNames) { + servers.add(sn.toString()); + } + + List conf = desc.getConfigurationList(); + byte[] data = new byte[0]; + if (conf.size() > 0) { + // Get backup root path + data = conf.get(0).getValue().getBytes(); + } + Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), data, servers); + if (proc == null) { + String msg = "Failed to submit distributed procedure for '" + desc.getInstance() + "'"; + LOG.error(msg); + throw new IOException(msg); + } + + try { + // wait for the procedure to complete. A timer thread is kicked off that should cancel this + // if it takes too long. + proc.waitForCompleted(); + LOG.info("Done waiting - exec procedure for " + desc.getInstance()); + LOG.info("Distributed roll log procedure is successful!"); + this.done = true; + } catch (InterruptedException e) { + ForeignException ee = + new ForeignException("Interrupted while waiting for roll log procdure to finish", e); + monitor.receive(ee); + Thread.currentThread().interrupt(); + } catch (ForeignException e) { + ForeignException ee = + new ForeignException("Exception while waiting for roll log procdure to finish", e); + monitor.receive(ee); + } + monitor.rethrowException(); + } + + private boolean isBackupEnabled() { + return BackupManager.isBackupEnabled(master.getConfiguration()); + } + + @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..8fc644c9fba --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.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.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.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.errorhandling.ForeignException; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.procedure.ProcedureMember; +import org.apache.hadoop.hbase.procedure.Subprocedure; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; +import org.apache.hadoop.hbase.regionserver.wal.FSHLog; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.WAL; + +/** + * This backup sub-procedure implementation forces a WAL rolling on a RS. + */ +@InterfaceAudience.Private +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; + private String backupRoot; + + public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member, + ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout, + LogRollBackupSubprocedurePool taskManager, byte[] data) { + + super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener, + wakeFrequency, timeout); + LOG.info("Constructing a LogRollBackupSubprocedure."); + this.rss = rss; + this.taskManager = taskManager; + if (data != null) { + backupRoot = new String(data); + } + } + + /** + * 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(); + List wals = rss.getWALs(); + long highest = -1; + for (WAL wal : wals) { + if (wal == null) continue; + if (((AbstractFSWAL) wal).getFilenum() > highest) { + highest = ((AbstractFSWAL) wal).getFilenum(); + } + } + + LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum + + " highest: " + highest + " on " + rss.getServerName()); + ((HRegionServer) rss).getWalRoller().requestRollAll(); + long start = EnvironmentEdgeManager.currentTime(); + while (!((HRegionServer) rss).getWalRoller().walRollFinished()) { + Thread.sleep(20); + } + LOG.debug("log roll took " + (EnvironmentEdgeManager.currentTime() - start)); + LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum() + + " on " + rss.getServerName()); + + Connection connection = rss.getConnection(); + try (final BackupSystemTable table = new BackupSystemTable(connection)) { + // sanity check, good for testing + HashMap serverTimestampMap = + table.readRegionServerLastLogRollResult(backupRoot); + String host = rss.getServerName().getHostname(); + int port = rss.getServerName().getPort(); + String server = host + ":" + port; + Long sts = serverTimestampMap.get(host); + if (sts != null && sts > highest) { + LOG.warn("Won't update server's last roll log result: current=" + sts + " new=" + highest); + return null; + } + // write the log number to backup system table. + table.writeRegionServerLastLogRollResult(server, highest, backupRoot); + return null; + } catch (Exception e) { + LOG.error(e); + throw e; + } + } + } + + private void rolllog() throws ForeignException { + monitor.rethrowException(); + + taskManager.submitTask(new RSRollLogTask()); + monitor.rethrowException(); + + // wait for everything to complete. + taskManager.waitForOutstandingTasks(); + monitor.rethrowException(); + + } + + @Override + public void acquireBarrier() throws ForeignException { + // do nothing, executing in inside barrier step. + } + + /** + * do a log roll. + * @return some bytes + */ + @Override + public byte[] insideBarrier() throws ForeignException { + rolllog(); + return null; + } + + /** + * Cancel threads if they haven't finished. + */ + @Override + public void cleanup(Exception e) { + taskManager.abort("Aborting log roll subprocedure tasks for backup due to error", e); + } + + /** + * Hooray! + */ + public void releaseBarrier() { + // NO OP + } + +} diff --git a/hbase-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..65a1fa346d6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java @@ -0,0 +1,139 @@ +/** + * 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.classification.InterfaceAudience; +import org.apache.hadoop.hbase.errorhandling.ForeignException; + +/** + * Handle running each of the individual tasks for completing a backup procedure on a region + * server. + */ +@InterfaceAudience.Private +public class LogRollBackupSubprocedurePool implements Closeable, Abortable { + private static final 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..9d5a8585761 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java @@ -0,0 +1,185 @@ +/** + * 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.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +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.RegionServerServices; +import org.apache.zookeeper.KeeperException; + +/** + * This manager class handles the work dealing with distributed WAL roll request. + *

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

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

+ * On shutdown, requires org.apache.hadoop.hbase.procedure.ProcedureMember.close() to be called + */ +@InterfaceAudience.Private +public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager { + + private static final Log LOG = LogFactory.getLog(LogRollRegionServerProcedureManager.class); + + /** Conf key for number of request threads to start backup on region servers */ + public static final String BACKUP_REQUEST_THREADS_KEY = "hbase.backup.region.pool.threads"; + /** # of threads for backup work on the rs. */ + public static final int BACKUP_REQUEST_THREADS_DEFAULT = 10; + + public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.timeout"; + public static final long BACKUP_TIMEOUT_MILLIS_DEFAULT = 60000; + + /** Conf key for millis between checks to see if backup work completed or if there are errors */ + public static final String BACKUP_REQUEST_WAKE_MILLIS_KEY = "hbase.backup.region.wakefrequency"; + /** Default amount of time to check for errors while regions finish backup work */ + private static final long BACKUP_REQUEST_WAKE_MILLIS_DEFAULT = 500; + + private RegionServerServices rss; + private ProcedureMemberRpcs memberRpcs; + private ProcedureMember member; + private boolean started = false; + + /** + * Create a default backup procedure manager + */ + public LogRollRegionServerProcedureManager() { + } + + /** + * Start accepting backup procedure requests. + */ + @Override + public void start() { + if (!BackupManager.isBackupEnabled(rss.getConfiguration())) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + this.memberRpcs.start(rss.getServerName().toString(), member); + started = true; + LOG.info("Started region server backup manager."); + } + + /** + * Close this and all running backup procedure tasks + * @param force forcefully stop all running tasks + * @throws IOException exception + */ + @Override + public void stop(boolean force) throws IOException { + if (!started) { + return; + } + String mode = force ? "abruptly" : "gracefully"; + LOG.info("Stopping RegionServerBackupManager " + mode + "."); + + try { + this.member.close(); + } finally { + this.memberRpcs.close(); + } + } + + /** + * If in a running state, creates the specified subprocedure for handling a backup procedure. + * @return Subprocedure to submit to the ProcedureMemeber. + */ + public Subprocedure buildSubprocedure(byte[] data) { + + // don't run a backup if the parent is stop(ping) + if (rss.isStopping() || rss.isStopped()) { + throw new IllegalStateException("Can't start backup procedure on RS: " + rss.getServerName() + + ", because stopping/stopped!"); + } + + LOG.info("Attempting to run a roll log procedure for backup."); + ForeignExceptionDispatcher errorDispatcher = new ForeignExceptionDispatcher(); + Configuration conf = rss.getConfiguration(); + long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT); + long wakeMillis = + conf.getLong(BACKUP_REQUEST_WAKE_MILLIS_KEY, BACKUP_REQUEST_WAKE_MILLIS_DEFAULT); + + LogRollBackupSubprocedurePool taskManager = + new LogRollBackupSubprocedurePool(rss.getServerName().toString(), conf); + return new LogRollBackupSubprocedure(rss, member, errorDispatcher, wakeMillis, timeoutMillis, + taskManager, data); + + } + + /** + * Build the actual backup procedure runner that will do all the 'hard' work + */ + public class BackupSubprocedureBuilder implements SubprocedureFactory { + + @Override + public Subprocedure buildSubprocedure(String name, byte[] data) { + return LogRollRegionServerProcedureManager.this.buildSubprocedure(data); + } + } + + @Override + public void initialize(RegionServerServices rss) throws KeeperException { + this.rss = rss; + if (!BackupManager.isBackupEnabled(rss.getConfiguration())) { + LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY + + " setting"); + return; + } + 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/backup/util/BackupSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java new file mode 100644 index 00000000000..0da6fc4ea13 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup.util; + +import java.util.List; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Backup set is a named group of HBase tables, which are managed together by Backup/Restore + * framework. Instead of using list of tables in backup or restore operation, one can use set's name + * instead. + */ +@InterfaceAudience.Private +public class BackupSet { + private final String name; + private final List tables; + + public BackupSet(String name, List tables) { + this.name = name; + this.tables = tables; + } + + public String getName() { + return name; + } + + public List getTables() { + return tables; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(name).append("={"); + sb.append(StringUtils.join(tables, ',')); + sb.append("}"); + return sb.toString(); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java new file mode 100644 index 00000000000..e32853d8092 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -0,0 +1,702 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.util; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.TreeMap; +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.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupRestoreConstants; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +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.AbstractFSWALProvider; + +/** + * A collection for methods used by multiple classes to backup HBase tables. + */ +@InterfaceAudience.Private +public final class BackupUtils { + protected static final Log LOG = LogFactory.getLog(BackupUtils.class); + public static final String LOGNAME_SEPARATOR = "."; + + private BackupUtils() { + throw new AssertionError("Instantiating utility class..."); + } + + /** + * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp + * value for the RS among the tables. + * @param rsLogTimestampMap timestamp map + * @return the min timestamp of each RS + */ + public static HashMap getRSLogTimestampMins( + HashMap> rsLogTimestampMap) { + + if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) { + return null; + } + + HashMap rsLogTimestampMins = new HashMap(); + HashMap> rsLogTimestampMapByRS = + new HashMap>(); + + for (Entry> tableEntry : rsLogTimestampMap.entrySet()) { + TableName table = tableEntry.getKey(); + HashMap rsLogTimestamp = tableEntry.getValue(); + for (Entry rsEntry : rsLogTimestamp.entrySet()) { + String rs = rsEntry.getKey(); + Long 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 (Entry> entry : rsLogTimestampMapByRS.entrySet()) { + String rs = entry.getKey(); + rsLogTimestampMins.put(rs, BackupUtils.getMinValue(entry.getValue())); + } + + return rsLogTimestampMins; + } + + /** + * copy out Table RegionInfo into incremental backup image need to consider move this + * logic into HBackupFileSystem + * @param conn connection + * @param backupInfo backup info + * @param conf configuration + * @throws IOException exception + * @throws InterruptedException exception + */ + public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo, + 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 (TableName table : backupInfo.getTables()) { + + if (!MetaTableAccessor.tableExists(conn, table)) { + LOG.warn("Table " + table + " does not exists, skipping it."); + continue; + } + HTableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table); + + // write a copy of descriptor to the target directory + Path target = new Path(backupInfo.getTableBackupDir(table)); + FileSystem targetFs = target.getFileSystem(conf); + FSTableDescriptors descriptors = + new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf)); + descriptors.createTableDescriptorForTableDirectory(target, orig, false); + LOG.debug("Attempting to copy table info for:" + table + " target: " + target + + " descriptor: " + orig); + LOG.debug("Finished copying tableinfo."); + List regions = null; + regions = MetaTableAccessor.getTableRegions(conn, table); + // For each region, write the region info to disk + LOG.debug("Starting to write region info for table " + table); + for (HRegionInfo regionInfo : regions) { + Path regionDir = + HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)), + regionInfo); + regionDir = + new Path(backupInfo.getTableBackupDir(table), regionDir.getName()); + writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo); + } + LOG.debug("Finished writing region info for table " + table); + } + } + + /** + * Write the .regioninfo file on-disk. + */ + public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs, + final Path regionInfoDir, HRegionInfo regionInfo) throws IOException { + final byte[] content = regionInfo.toDelimitedByteArray(); + Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR); + // 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(); + } + } + + /** + * Parses hostname:port from WAL file path + * @param p path to WAL file + * @return hostname:port + */ + public static String parseHostNameFromLogFile(Path p) { + try { + if (AbstractFSWALProvider.isArchivedLogFile(p)) { + return BackupUtils.parseHostFromOldLog(p); + } else { + ServerName sname = AbstractFSWALProvider.getServerNameFromWALDirectoryName(p); + if (sname != null) { + return sname.getAddress().toString(); + } else { + LOG.error("Skip log file (can't parse): " + p); + return null; + } + } + } catch (Exception e) { + LOG.error("Skip log file (can't parse): " + p, e); + return null; + } + } + + /** + * Returns WAL file name + * @param walFileName WAL file name + * @return WAL file name + * @throws IOException exception + * @throws IllegalArgumentException exception + */ + public static String getUniqueWALFileNamePart(String walFileName) throws IOException { + return getUniqueWALFileNamePart(new Path(walFileName)); + } + + /** + * Returns 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(); + } + + /** + * 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.isDirectory()) { + totalLength += getFilesLength(fs, fileStatus.getPath()); + } else { + totalLength += fileStatus.getLen(); + } + } + } + return totalLength; + } + + /** + * Get list of all old WAL files (WALs and archive) + * @param c configuration + * @param hostTimestampMap {host,timestamp} map + * @return list of WAL files + * @throws IOException exception + */ + public static List getWALFilesOlderThan(final Configuration c, + final HashMap hostTimestampMap) throws IOException { + Path 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 (AbstractFSWALProvider.isMetaFile(p)) { + return false; + } + String host = parseHostNameFromLogFile(p); + if (host == null) { + return false; + } + Long oldTimestamp = hostTimestampMap.get(host); + Long currentLogTS = BackupUtils.getCreationTime(p); + return currentLogTS <= oldTimestamp; + } catch (Exception e) { + LOG.warn("Can not parse" + p, e); + return false; + } + } + }; + FileSystem fs = FileSystem.get(c); + logFiles = BackupUtils.getFiles(fs, logDir, logFiles, filter); + logFiles = BackupUtils.getFiles(fs, oldLogDir, logFiles, filter); + return logFiles; + } + + public static TableName[] parseTableNames(String tables) { + if (tables == null) { + return null; + } + String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + + TableName[] ret = new TableName[tableArray.length]; + for (int i = 0; i < tableArray.length; i++) { + ret[i] = TableName.valueOf(tableArray[i]); + } + return ret; + } + + + /** + * Check whether the backup path exist + * @param backupStr backup + * @param conf configuration + * @return Yes if path exists + * @throws IOException exception + */ + public static boolean checkPathExist(String backupStr, Configuration conf) throws IOException { + boolean isExist = false; + Path backupPath = new Path(backupStr); + FileSystem fileSys = backupPath.getFileSystem(conf); + String targetFsScheme = fileSys.getUri().getScheme(); + if (LOG.isTraceEnabled()) { + LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme); + } + if (fileSys.exists(backupPath)) { + isExist = true; + } + return isExist; + } + + /** + * Check target path first, confirm it doesn't exist before backup + * @param backupRootPath backup destination path + * @param conf configuration + * @throws IOException exception + */ + public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException { + boolean targetExists = false; + try { + targetExists = checkPathExist(backupRootPath, conf); + } catch (IOException e) { + String expMsg = e.getMessage(); + String newMsg = null; + if (expMsg.contains("No FileSystem for scheme")) { + newMsg = + "Unsupported filesystem scheme found in the backup target url. Error Message: " + + newMsg; + LOG.error(newMsg); + throw new IOException(newMsg); + } else { + throw e; + } + } + + if (targetExists) { + LOG.info("Using existing backup root dir: " + backupRootPath); + } else { + LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created."); + } + } + + /** + * Get the min value for all the Values a map. + * @param map map + * @return the min value + */ + public static Long getMinValue(HashMap map) { + Long minTimestamp = null; + if (map != null) { + ArrayList timestampList = new ArrayList(map.values()); + Collections.sort(timestampList); + // The min among all the RS log timestamps will be kept in backup system table table. + minTimestamp = timestampList.get(0); + } + return minTimestamp; + } + + /** + * Parses host name:port from archived WAL path + * @param p path + * @return host name + * @throws IOException exception + */ + public static String parseHostFromOldLog(Path p) { + try { + String n = p.getName(); + int idx = n.lastIndexOf(LOGNAME_SEPARATOR); + String s = URLDecoder.decode(n.substring(0, idx), "UTF8"); + return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s); + } catch (Exception e) { + LOG.warn("Skip log file (can't parse): " + p); + return null; + } + } + + /** + * Given the log file, parse the timestamp from the file name. The timestamp is the last number. + * @param p a path to the log file + * @return the timestamp + * @throws IOException exception + */ + public static Long getCreationTime(Path p) throws IOException { + int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR); + if (idx < 0) { + throw new IOException("Cannot parse timestamp from path " + p); + } + String ts = p.getName().substring(idx + 1); + return Long.parseLong(ts); + } + + public static List getFiles(FileSystem fs, Path rootDir, List files, + PathFilter filter) throws 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()); + } + } + return files; + } + + public static void cleanupBackupData(BackupInfo context, Configuration conf) throws IOException { + cleanupHLogDir(context, conf); + cleanupTargetDir(context, conf); + } + + /** + * Clean up directories which are generated when DistCp copying hlogs + * @param backupInfo backup info + * @param conf configuration + * @throws IOException exception + */ + private static void cleanupHLogDir(BackupInfo backupInfo, Configuration conf) + throws IOException { + + String logDir = backupInfo.getHLogTargetDir(); + if (logDir == null) { + LOG.warn("No log directory specified for " + backupInfo.getBackupId()); + return; + } + + Path rootPath = new Path(logDir).getParent(); + FileSystem fs = FileSystem.get(rootPath.toUri(), conf); + FileStatus[] files = listStatus(fs, rootPath, null); + if (files == null) { + return; + } + for (FileStatus file : files) { + LOG.debug("Delete log files: " + file.getPath().getName()); + fs.delete(file.getPath(), true); + } + } + + + private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) { + try { + // clean up the data at target directory + LOG.debug("Trying to cleanup up target dir : " + backupInfo.getBackupId()); + String targetDir = backupInfo.getBackupRootDir(); + if (targetDir == null) { + LOG.warn("No target directory specified for " + backupInfo.getBackupId()); + return; + } + + FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + + for (TableName table : backupInfo.getTables()) { + Path targetDirPath = + new Path(getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(), + table)); + if (outputFs.delete(targetDirPath, true)) { + LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done."); + } else { + LOG.info("No data has been found in " + targetDirPath.toString() + "."); + } + + Path tableDir = targetDirPath.getParent(); + FileStatus[] backups = listStatus(outputFs, tableDir, null); + if (backups == null || backups.length == 0) { + outputFs.delete(tableDir, true); + LOG.debug(tableDir.toString() + " is empty, remove it."); + } + } + outputFs.delete(new Path(targetDir, backupInfo.getBackupId()), true); + } catch (IOException e1) { + LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " at " + + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + "."); + } + } + + /** + * Given the backup root dir, backup id and the table name, return the backup image location, + * which is also where the backup manifest file is. return value look like: + * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/" + * @param backupRootDir backup root directory + * @param backupId backup id + * @param tableName table name + * @return backupPath String for the particular table + */ + public static String getTableBackupDir(String backupRootDir, String backupId, + TableName tableName) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString() + + Path.SEPARATOR; + } + + /** + * Sort history list by start time in descending order. + * @param historyList history list + * @return sorted list of BackupCompleteData + */ + public static ArrayList sortHistoryListDesc(ArrayList historyList) { + ArrayList list = new ArrayList(); + TreeMap map = new TreeMap(); + for (BackupInfo h : historyList) { + map.put(Long.toString(h.getStartTs()), h); + } + Iterator i = map.descendingKeySet().iterator(); + while (i.hasNext()) { + list.add(map.get(i.next())); + } + return list; + } + + + /** + * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates + * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and + * return an empty FileStatus[] while Hadoop 2 will throw FileNotFoundException. + * @param fs file system + * @param dir directory + * @param filter path filter + * @return null if dir is empty or doesn't exist, otherwise FileStatus array + */ + public static FileStatus[] + listStatus(final FileSystem fs, final Path dir, final PathFilter filter) throws IOException { + FileStatus[] status = null; + try { + status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter); + } catch (FileNotFoundException fnfe) { + // if directory doesn't exist, return null + if (LOG.isTraceEnabled()) { + LOG.trace(dir + " doesn't exist"); + } + } + if (status == null || status.length < 1) return null; + return status; + } + + /** + * Return the 'path' component of a Path. In Hadoop, Path is an URI. This method returns the + * 'path' component of a Path's URI: e.g. If a Path is + * hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir, this method returns + * /hbase_trunk/TestTable/compaction.dir. This method is useful if you want to print + * out a Path without qualifying Filesystem instance. + * @param p file system Path whose 'path' component we are to return. + * @return Path portion of the Filesystem + */ + public static String getPath(Path p) { + return p.toUri().getPath(); + } + + /** + * Given the backup root dir and the backup id, return the log file location for an incremental + * backup. + * @param backupRootDir backup root directory + * @param backupId backup id + * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738" + */ + public static String getLogBackupDir(String backupRootDir, String backupId) { + return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR + + HConstants.HREGION_LOGDIR_NAME; + } + + private static List getHistory(Configuration conf, Path backupRootPath) + throws IOException { + // Get all (n) history from backup root destination + FileSystem fs = FileSystem.get(conf); + RemoteIterator it = fs.listLocatedStatus(backupRootPath); + + List infos = new ArrayList(); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (!lfs.isDirectory()) continue; + String backupId = lfs.getPath().getName(); + try { + BackupInfo info = loadBackupInfo(backupRootPath, backupId, fs); + infos.add(info); + } catch (IOException e) { + LOG.error("Can not load backup info from: " + lfs.getPath(), e); + } + } + // Sort + Collections.sort(infos, new Comparator() { + + @Override + public int compare(BackupInfo o1, BackupInfo o2) { + long ts1 = getTimestamp(o1.getBackupId()); + long ts2 = getTimestamp(o2.getBackupId()); + if (ts1 == ts2) return 0; + return ts1 < ts2 ? 1 : -1; + } + + private long getTimestamp(String backupId) { + String[] split = backupId.split("_"); + return Long.parseLong(split[1]); + } + }); + return infos; + } + + public static List getHistory(Configuration conf, int n, Path backupRootPath, + BackupInfo.Filter... filters) throws IOException { + List infos = getHistory(conf, backupRootPath); + List ret = new ArrayList(); + for (BackupInfo info : infos) { + if (ret.size() == n) { + break; + } + boolean passed = true; + for (int i = 0; i < filters.length; i++) { + if (!filters[i].apply(info)) { + passed = false; + break; + } + } + if (passed) { + ret.add(info); + } + } + return ret; + } + + public static BackupInfo loadBackupInfo(Path backupRootPath, String backupId, FileSystem fs) + throws IOException { + Path backupPath = new Path(backupRootPath, backupId); + + RemoteIterator it = fs.listFiles(backupPath, true); + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (lfs.getPath().getName().equals(BackupManifest.MANIFEST_FILE_NAME)) { + // Load BackupManifest + BackupManifest manifest = new BackupManifest(fs, lfs.getPath().getParent()); + BackupInfo info = manifest.toBackupInfo(); + return info; + } + } + return null; + } + + /** + * Create restore request. + * @param backupRootDir backup root dir + * @param backupId backup id + * @param check check only + * @param fromTables table list from + * @param toTables table list to + * @param isOverwrite overwrite data + * @return request obkect + */ + public static RestoreRequest createRestoreRequest(String backupRootDir, String backupId, + boolean check, TableName[] fromTables, TableName[] toTables, boolean isOverwrite) { + RestoreRequest.Builder builder = new RestoreRequest.Builder(); + RestoreRequest request = builder.withBackupRootDir(backupRootDir) + .withBackupId(backupId) + .withCheck(check) + .withFromTables(fromTables) + .withToTables(toTables) + .withOvewrite(isOverwrite).build(); + return request; + } + + public static boolean validate(HashMap backupManifestMap, + Configuration conf) throws IOException { + boolean isValid = true; + + for (Entry manifestEntry : backupManifestMap.entrySet()) { + TableName table = manifestEntry.getKey(); + TreeSet imageSet = new TreeSet(); + + ArrayList depList = manifestEntry.getValue().getDependentListByTable(table); + if (depList != null && !depList.isEmpty()) { + imageSet.addAll(depList); + } + + LOG.info("Dependent image(s) from old to new:"); + for (BackupImage image : imageSet) { + String imageDir = + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table); + if (!BackupUtils.checkPathExist(imageDir, conf)) { + LOG.error("ERROR: backup image does not exist: " + imageDir); + isValid = false; + break; + } + LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available"); + } + } + return isValid; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java new file mode 100644 index 00000000000..a130c213f9e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java @@ -0,0 +1,610 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.util; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.TreeMap; + +import org.apache.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.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.BackupRestoreFactory; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.RestoreJob; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles; +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.shaded.protobuf.generated.HBaseProtos.SnapshotDescription; +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.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSTableDescriptors; + +/** + * A collection for methods used by multiple classes to restore HBase tables. + */ +@InterfaceAudience.Private +public class RestoreTool { + + public static final Log LOG = LogFactory.getLog(BackupUtils.class); + + private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR }; + + private final static long TABLE_AVAILABILITY_WAIT_TIME = 180000; + + protected Configuration conf = null; + + protected Path backupRootPath; + + protected String backupId; + + protected FileSystem fs; + private final Path restoreTmpPath; + + // store table name and snapshot dir mapping + private final HashMap snapshotMap = new HashMap<>(); + + public RestoreTool(Configuration conf, final Path backupRootPath, final String backupId) + throws IOException { + this.conf = conf; + this.backupRootPath = backupRootPath; + this.backupId = backupId; + this.fs = backupRootPath.getFileSystem(conf); + this.restoreTmpPath = + new Path(conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, + HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY), "restore"); + } + + /** + * 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 + */ + Path getTableArchivePath(TableName tableName) throws IOException { + + Path baseDir = + new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), + HConstants.HFILE_ARCHIVE_DIRECTORY); + Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR); + Path archivePath = new Path(dataDir, tableName.getNamespaceAsString()); + Path tableArchivePath = new Path(archivePath, tableName.getQualifierAsString()); + if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) { + LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists"); + tableArchivePath = null; // empty table has no archive + } + return tableArchivePath; + } + + /** + * Gets region list + * @param tableName table name + * @return RegionList region list + * @throws FileNotFoundException exception + * @throws IOException exception + */ + ArrayList getRegionList(TableName tableName) throws FileNotFoundException, IOException { + Path tableArchivePath = getTableArchivePath(tableName); + ArrayList regionDirList = new ArrayList(); + FileStatus[] children = fs.listStatus(tableArchivePath); + for (FileStatus childStatus : children) { + // here child refer to each region(Name) + Path child = childStatus.getPath(); + regionDirList.add(child); + } + return regionDirList; + } + + + void modifyTableSync(Connection conn, HTableDescriptor desc) throws IOException { + + try (Admin admin = conn.getAdmin();) { + admin.modifyTable(desc.getTableName(), desc); + int attempt = 0; + int maxAttempts = 600; + while (!admin.isTableAvailable(desc.getTableName())) { + Thread.sleep(100); + attempt++; + if (attempt++ > maxAttempts) { + throw new IOException("Timeout expired " + (maxAttempts * 100) + "ms"); + } + } + } catch (Exception e) { + throw new IOException(e); + } + } + + /** + * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently + * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in + * the future + * @param conn HBase connection + * @param tableBackupPath backup path + * @param logDirs : incremental backup folders, which contains WAL + * @param tableNames : source tableNames(table names were backuped) + * @param newTableNames : target tableNames(table names to be restored to) + * @param incrBackupId incremental backup Id + * @throws IOException exception + */ + public void incrementalRestoreTable(Connection conn, Path tableBackupPath, Path[] logDirs, + TableName[] tableNames, TableName[] newTableNames, String incrBackupId) throws IOException { + + try (Admin admin = conn.getAdmin();) { + if (tableNames.length != newTableNames.length) { + throw new IOException("Number of source tables and target tables does not match!"); + } + FileSystem fileSys = tableBackupPath.getFileSystem(this.conf); + + // for incremental backup image, expect the table already created either by user or previous + // full backup. Here, check that all new tables exists + for (TableName tableName : newTableNames) { + if (!admin.tableExists(tableName)) { + throw new IOException("HBase table " + tableName + + " does not exist. Create the table first, e.g. by restoring a full backup."); + } + } + // adjust table schema + for (int i = 0; i < tableNames.length; i++) { + TableName tableName = tableNames[i]; + HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId); + LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId); + + TableName newTableName = newTableNames[i]; + HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName); + List families = Arrays.asList(tableDescriptor.getColumnFamilies()); + List existingFamilies = + Arrays.asList(newTableDescriptor.getColumnFamilies()); + boolean schemaChangeNeeded = false; + for (HColumnDescriptor family : families) { + if (!existingFamilies.contains(family)) { + newTableDescriptor.addFamily(family); + schemaChangeNeeded = true; + } + } + for (HColumnDescriptor family : existingFamilies) { + if (!families.contains(family)) { + newTableDescriptor.removeFamily(family.getName()); + schemaChangeNeeded = true; + } + } + if (schemaChangeNeeded) { + modifyTableSync(conn, newTableDescriptor); + LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor); + } + } + RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf); + + restoreService.run(logDirs, tableNames, newTableNames, false); + } + } + + public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName, + TableName newTableName, boolean truncateIfExists, String lastIncrBackupId) + throws IOException { + restoreTableAndCreate(conn, tableName, newTableName, tableBackupPath, truncateIfExists, + lastIncrBackupId); + } + + /** + * Returns value represent path for path to backup table snapshot directory: + * "/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot" + * @param backupRootPath backup root path + * @param tableName table name + * @param backupId backup Id + * @return path for snapshot + */ + Path getTableSnapshotPath(Path backupRootPath, TableName tableName, String backupId) { + return new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId), + HConstants.SNAPSHOT_DIR_NAME); + } + + /** + * Returns value represent path for: + * ""/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot/snapshot_1396650097621_namespace_table" + * this path contains .snapshotinfo, .tabledesc (0.96 and 0.98) this path contains .snapshotinfo, + * .data.manifest (trunk) + * @param tableName table name + * @return path to table info + * @throws FileNotFoundException exception + * @throws IOException exception + */ + Path getTableInfoPath(TableName tableName) throws FileNotFoundException, IOException { + Path tableSnapShotPath = getTableSnapshotPath(backupRootPath, tableName, backupId); + Path tableInfoPath = null; + + // can't build the path directly as the timestamp values are different + FileStatus[] snapshots = fs.listStatus(tableSnapShotPath); + for (FileStatus snapshot : snapshots) { + tableInfoPath = snapshot.getPath(); + // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest"; + if (tableInfoPath.getName().endsWith("data.manifest")) { + break; + } + } + return tableInfoPath; + } + + /** + * Get table descriptor + * @param tableName is the table backed up + * @return {@link HTableDescriptor} saved in backup image of the table + */ + HTableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException { + Path tableInfoPath = this.getTableInfoPath(tableName); + SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath); + SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc); + HTableDescriptor tableDescriptor = manifest.getTableDescriptor(); + if (!tableDescriptor.getTableName().equals(tableName)) { + LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: " + + tableInfoPath.toString()); + LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString()); + throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName + + " under tableInfoPath: " + tableInfoPath.toString()); + } + return tableDescriptor; + } + + /** + * 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 + */ + 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"); + if (desFs.exists(restoreTmpPath)) { + try { + desFs.delete(restoreTmpPath, true); + } catch (IOException e) { + LOG.debug("Failed to delete path: " + restoreTmpPath + + ", need to check whether restore target DFS cluster is healthy"); + } + } + FileUtil.copy(srcFs, tableArchivePath, desFs, restoreTmpPath, false, conf); + LOG.debug("Copied to temporary path on local cluster: " + restoreTmpPath); + tableArchivePath = restoreTmpPath; + } + return tableArchivePath; + } + + private HTableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName, + String lastIncrBackupId) throws IOException { + if (lastIncrBackupId != null) { + String target = + BackupUtils.getTableBackupDir(backupRootPath.toString(), + lastIncrBackupId, tableName); + return FSTableDescriptors.getTableDescriptorFromFs(fileSys, new Path(target)); + } + return null; + } + + private void restoreTableAndCreate(Connection conn, TableName tableName, TableName newTableName, + Path tableBackupPath, boolean truncateIfExists, String lastIncrBackupId) throws IOException { + if (newTableName == null) { + newTableName = tableName; + } + FileSystem fileSys = tableBackupPath.getFileSystem(this.conf); + + // get table descriptor first + HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId); + if (tableDescriptor != null) { + LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId); + } + + if (tableDescriptor == null) { + Path tableSnapshotPath = getTableSnapshotPath(backupRootPath, tableName, backupId); + if (fileSys.exists(tableSnapshotPath)) { + // snapshot path exist means the backup path is in HDFS + // check whether snapshot dir already recorded for target table + if (snapshotMap.get(tableName) != null) { + SnapshotDescription desc = + SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath); + SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc); + tableDescriptor = manifest.getTableDescriptor(); + } else { + tableDescriptor = getTableDesc(tableName); + snapshotMap.put(tableName, getTableInfoPath(tableName)); + } + if (tableDescriptor == null) { + LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost"); + } + } else { + throw new IOException("Table snapshot directory: " + + tableSnapshotPath + " does not exist."); + } + } + + Path tableArchivePath = getTableArchivePath(tableName); + if (tableArchivePath == null) { + if (tableDescriptor != null) { + // find table descriptor but no archive dir means the table is empty, create table and exit + if (LOG.isDebugEnabled()) { + LOG.debug("find table descriptor but no archive dir for table " + tableName + + ", will only create table"); + } + tableDescriptor.setName(newTableName); + checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor, + truncateIfExists); + return; + } else { + throw new IllegalStateException("Cannot restore hbase table because directory '" + + " tableArchivePath is null."); + } + } + + if (tableDescriptor == null) { + tableDescriptor = new HTableDescriptor(newTableName); + } else { + tableDescriptor.setName(newTableName); + } + + // record all region dirs: + // load all files in dir + try { + ArrayList regionPathList = getRegionList(tableName); + + // should only try to create the table with all region informations, so we could pre-split + // the regions in fine grain + checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, regionPathList, + tableDescriptor, truncateIfExists); + if (tableArchivePath != null) { + // start real restore through bulkload + // if the backup target is on local cluster, special action needed + Path tempTableArchivePath = checkLocalAndBackup(tableArchivePath); + if (tempTableArchivePath.equals(tableArchivePath)) { + if (LOG.isDebugEnabled()) { + LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath); + } + } else { + regionPathList = getRegionList(tempTableArchivePath); // point to the tempDir + if (LOG.isDebugEnabled()) { + LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath); + } + } + + LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false); + for (Path regionPath : regionPathList) { + String regionName = regionPath.toString(); + if (LOG.isDebugEnabled()) { + LOG.debug("Restoring HFiles from directory " + regionName); + } + String[] args = { regionName, newTableName.getNameAsString() }; + loader.run(args); + } + } + // we do not recovered edits + } catch (Exception e) { + throw new IllegalStateException("Cannot restore hbase table", e); + } + } + + /** + * Gets region list + * @param tableArchivePath table archive path + * @return RegionList region list + * @throws FileNotFoundException exception + * @throws IOException exception + */ + 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; + } + + /** + * 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 { + + // By default, it is 32 and loader will fail if # of files in any region exceed this + // limit. Bad for snapshot restore. + this.conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE); + this.conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes"); + LoadIncrementalHFiles loader = null; + try { + loader = new LoadIncrementalHFiles(this.conf); + } catch (Exception e1) { + throw new IOException(e1); + } + return loader; + } + + /** + * Calculate region boundaries and add all the column families to the table descriptor + * @param regionDirList region dir list + * @return a set of keys to store the boundaries + */ + byte[][] generateBoundaryKeys(ArrayList regionDirList) throws FileNotFoundException, + IOException { + TreeMap map = new TreeMap(Bytes.BYTES_COMPARATOR); + // Build a set of keys to store the boundaries + // calculate region boundaries and add all the column families to the table descriptor + for (Path regionDir : regionDirList) { + LOG.debug("Parsing region dir: " + regionDir); + Path hfofDir = regionDir; + + if (!fs.exists(hfofDir)) { + LOG.warn("HFileOutputFormat dir " + hfofDir + " not found"); + } + + FileStatus[] familyDirStatuses = fs.listStatus(hfofDir); + if (familyDirStatuses == null) { + throw new IOException("No families found in " + hfofDir); + } + + for (FileStatus stat : familyDirStatuses) { + if (!stat.isDirectory()) { + LOG.warn("Skipping non-directory " + stat.getPath()); + continue; + } + boolean isIgnore = false; + String pathName = stat.getPath().getName(); + for (String ignore : ignoreDirs) { + if (pathName.contains(ignore)) { + LOG.warn("Skipping non-family directory" + pathName); + isIgnore = true; + break; + } + } + if (isIgnore) { + continue; + } + Path familyDir = stat.getPath(); + LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]"); + // Skip _logs, etc + if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) { + continue; + } + + // start to parse hfile inside one family dir + Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir)); + for (Path hfile : hfiles) { + if (hfile.getName().startsWith("_") || hfile.getName().startsWith(".") + || StoreFileInfo.isReference(hfile.getName()) + || HFileLink.isHFileLink(hfile.getName())) { + continue; + } + HFile.Reader reader = HFile.createReader(fs, hfile, conf); + final byte[] first, last; + try { + 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(); + } + } + } + } + return LoadIncrementalHFiles.inferBoundaries(map); + } + + /** + * Prepare the table for bulkload, most codes copied from + * {@link LoadIncrementalHFiles#createTable(String, String)} + * @param conn connection + * @param tableBackupPath path + * @param tableName table name + * @param targetTableName target table name + * @param regionDirList region directory list + * @param htd table descriptor + * @param truncateIfExists truncates table if exists + * @throws IOException exception + */ + private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName, + TableName targetTableName, ArrayList regionDirList, HTableDescriptor htd, + boolean truncateIfExists) throws IOException { + try (Admin admin = conn.getAdmin();) { + boolean createNew = false; + if (admin.tableExists(targetTableName)) { + if (truncateIfExists) { + LOG.info("Truncating exising target table '" + targetTableName + + "', preserving region splits"); + admin.disableTable(targetTableName); + admin.truncateTable(targetTableName, true); + } else { + LOG.info("Using exising target table '" + targetTableName + "'"); + } + } else { + createNew = true; + } + if (createNew) { + LOG.info("Creating target table '" + targetTableName + "'"); + byte[][] keys = null; + if (regionDirList == null || regionDirList.size() == 0) { + admin.createTable(htd, null); + } else { + keys = generateBoundaryKeys(regionDirList); + // create table using table descriptor and region boundaries + admin.createTable(htd, keys); + } + long startTime = EnvironmentEdgeManager.currentTime(); + while (!admin.isTableAvailable(targetTableName, keys)) { + try { + Thread.sleep(100); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) { + throw new IOException("Time out " + TABLE_AVAILABILITY_WAIT_TIME + "ms expired, table " + + targetTableName + " is still not available"); + } + } + } + } + } + +} diff --git a/hbase-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..0e3755bf993 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,9 +17,14 @@ */ package org.apache.hadoop.hbase.coordination; -import org.apache.hadoop.hbase.classification.InterfaceAudience; +import java.io.IOException; + import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs; +import org.apache.zookeeper.KeeperException; /** * Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations. @@ -51,8 +56,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.ProcedureMemberRpcs} + */ + public abstract ProcedureMemberRpcs + getProcedureMemberRpcs(String procType) throws KeeperException; + } 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..8ce5f9cd8e4 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,10 +17,17 @@ */ package org.apache.hadoop.hbase.coordination; -import org.apache.hadoop.hbase.classification.InterfaceAudience; +import java.io.IOException; + import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.Server; +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.procedure.ZKProcedureCoordinator; +import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; /** * ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}. @@ -49,9 +56,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 ZKProcedureCoordinator(watcher, procType, coordNode); + } + + @Override + public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws KeeperException { + return new ZKProcedureMemberRpcs(watcher, procType); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java new file mode 100644 index 00000000000..e90d5c1cb9d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java @@ -0,0 +1,174 @@ +/** + * 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.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple MR input format for HFiles. + * This code was borrowed from Apache Crunch project. + * Updated to the recent version of HBase. + */ +public class HFileInputFormat extends FileInputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(HFileInputFormat.class); + + /** + * File filter that removes all "hidden" files. This might be something worth removing from + * a more general purpose utility; it accounts for the presence of metadata files created + * in the way we're doing exports. + */ + static final PathFilter HIDDEN_FILE_FILTER = new PathFilter() { + @Override + public boolean accept(Path p) { + String name = p.getName(); + return !name.startsWith("_") && !name.startsWith("."); + } + }; + + /** + * Record reader for HFiles. + */ + private static class HFileRecordReader extends RecordReader { + + private Reader in; + protected Configuration conf; + private HFileScanner scanner; + + /** + * A private cache of the key value so it doesn't need to be loaded twice from the scanner. + */ + private Cell value = null; + private long count; + private boolean seeked = false; + + @Override + public void initialize(InputSplit split, TaskAttemptContext context) + throws IOException, InterruptedException { + FileSplit fileSplit = (FileSplit) split; + conf = context.getConfiguration(); + Path path = fileSplit.getPath(); + FileSystem fs = path.getFileSystem(conf); + LOG.info("Initialize HFileRecordReader for {}", path); + this.in = HFile.createReader(fs, path, conf); + + // The file info must be loaded before the scanner can be used. + // This seems like a bug in HBase, but it's easily worked around. + this.in.loadFileInfo(); + this.scanner = in.getScanner(false, false); + + } + + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + boolean hasNext; + if (!seeked) { + LOG.info("Seeking to start"); + hasNext = scanner.seekTo(); + seeked = true; + } else { + hasNext = scanner.next(); + } + if (!hasNext) { + return false; + } + value = scanner.getCell(); + count++; + return true; + } + + @Override + public NullWritable getCurrentKey() throws IOException, InterruptedException { + return NullWritable.get(); + } + + @Override + public Cell getCurrentValue() throws IOException, InterruptedException { + return value; + } + + @Override + public float getProgress() throws IOException, InterruptedException { + // This would be inaccurate if KVs are not uniformly-sized or we have performed a seek to + // the start row, but better than nothing anyway. + return 1.0f * count / in.getEntries(); + } + + @Override + public void close() throws IOException { + if (in != null) { + in.close(); + in = null; + } + } + } + + @Override + protected List listStatus(JobContext job) throws IOException { + List result = new ArrayList(); + + // Explode out directories that match the original FileInputFormat filters + // since HFiles are written to directories where the + // directory name is the column name + for (FileStatus status : super.listStatus(job)) { + if (status.isDirectory()) { + FileSystem fs = status.getPath().getFileSystem(job.getConfiguration()); + for (FileStatus match : fs.listStatus(status.getPath(), HIDDEN_FILE_FILTER)) { + result.add(match); + } + } else { + result.add(status); + } + } + return result; + } + + @Override + public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) + throws IOException, InterruptedException { + return new HFileRecordReader(); + } + + @Override + protected boolean isSplitable(JobContext context, Path filename) { + // This file isn't splittable. + return false; + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 718e88bc4e2..f59e24ce7a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -100,7 +100,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; import com.google.common.util.concurrent.ThreadFactoryBuilder; - /** * Tool to load the output of HFileOutputFormat into an existing table. */ @@ -116,7 +115,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { = "hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily"; private static final String ASSIGN_SEQ_IDS = "hbase.mapreduce.bulkload.assign.sequenceNumbers"; public final static String CREATE_TABLE_CONF_KEY = "create.table"; - public final static String SILENCE_CONF_KEY = "ignore.unmatched.families"; + public final static String IGNORE_UNMATCHED_CF_CONF_KEY = "ignore.unmatched.families"; public final static String ALWAYS_COPY_FILES = "always.copy.files"; // We use a '.' prefix which is ignored when walking directory trees @@ -168,7 +167,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { System.err.println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename" + "\n -D" + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n" + " Note: if you set this to 'no', then the target table must already exist in HBase\n -D" - + SILENCE_CONF_KEY + "=yes - can be used to ignore unmatched column families\n" + + IGNORE_UNMATCHED_CF_CONF_KEY + "=yes - can be used to ignore unmatched column families\n" + "\n"); } @@ -530,7 +529,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool { boolean success = false; try { LOG.debug("Going to connect to server " + getLocation() + " for row " - + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths); + + Bytes.toStringBinary(getRow()) + " with hfile group " + + LoadIncrementalHFiles.this.toString( famPaths)); byte[] regionName = getLocation().getRegionInfo().getRegionName(); try (Table table = conn.getTable(getTableName())) { secureClient = new SecureBulkLoadClient(getConf(), table); @@ -1047,6 +1047,21 @@ public class LoadIncrementalHFiles extends Configured implements Tool { } } + private final String toString(List> list) { + StringBuffer sb = new StringBuffer(); + sb.append("["); + if(list != null){ + for(Pair pair: list) { + sb.append("{"); + sb.append(Bytes.toStringBinary(pair.getFirst())); + sb.append(","); + sb.append(pair.getSecond()); + sb.append("}"); + } + } + sb.append("]"); + return sb.toString(); + } private boolean isSecureBulkLoadEndpointAvailable() { String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ""); return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint"); @@ -1245,7 +1260,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { try (Table table = connection.getTable(tableName); RegionLocator locator = connection.getRegionLocator(tableName)) { - boolean silence = "yes".equalsIgnoreCase(getConf().get(SILENCE_CONF_KEY, "")); + boolean silence = "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, "")); boolean copyFiles = "yes".equalsIgnoreCase(getConf().get(ALWAYS_COPY_FILES, "")); if (dirPath != null) { doBulkLoad(hfofDir, admin, table, locator, silence, copyFiles); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java index 8514ace0a73..0ca78b42731 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java @@ -27,22 +27,26 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.StringUtils; /** * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} files. @@ -169,7 +173,7 @@ public class WALInputFormat extends InputFormat { temp = reader.next(currentEntry); i++; } catch (EOFException x) { - LOG.info("Corrupted entry detected. Ignoring the rest of the file." + LOG.warn("Corrupted entry detected. Ignoring the rest of the file." + " (This is normal when a RegionServer crashed.)"); return false; } @@ -231,29 +235,37 @@ public class WALInputFormat extends InputFormat { List getSplits(final JobContext context, final String startKey, final String endKey) throws IOException, InterruptedException { Configuration conf = context.getConfiguration(); - Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir")); - + Path[] inputPaths = getInputPaths(conf); long startTime = conf.getLong(startKey, Long.MIN_VALUE); long endTime = conf.getLong(endKey, Long.MAX_VALUE); - FileSystem fs = inputDir.getFileSystem(conf); - List files = getFiles(fs, inputDir, startTime, endTime); - - List splits = new ArrayList<>(files.size()); - for (FileStatus file : files) { + List allFiles = new ArrayList(); + for(Path inputPath: inputPaths){ + FileSystem fs = inputPath.getFileSystem(conf); + List files = getFiles(fs, inputPath, startTime, endTime); + allFiles.addAll(files); + } + List splits = new ArrayList(allFiles.size()); + for (FileStatus file : allFiles) { splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime)); } return splits; } + private Path[] getInputPaths(Configuration conf) { + String inpDirs = conf.get("mapreduce.input.fileinputformat.inputdir"); + return StringUtils.stringToPath(inpDirs.split(",")); + } + private List getFiles(FileSystem fs, Path dir, long startTime, long endTime) throws IOException { List result = new ArrayList<>(); LOG.debug("Scanning " + dir.toString() + " for WAL files"); - FileStatus[] files = fs.listStatus(dir); - if (files == null) return Collections.emptyList(); - for (FileStatus file : files) { + RemoteIterator iter = fs.listLocatedStatus(dir); + if (!iter.hasNext()) return Collections.emptyList(); + while (iter.hasNext()) { + LocatedFileStatus file = iter.next(); if (file.isDirectory()) { // recurse into sub directories result.addAll(getFiles(fs, file.getPath(), startTime, endTime)); @@ -264,7 +276,7 @@ public class WALInputFormat extends InputFormat { try { long fileStartTime = Long.parseLong(name.substring(idx+1)); if (fileStartTime <= endTime) { - LOG.info("Found: " + name); + LOG.info("Found: " + file); result.add(file); } } catch (NumberFormatException x) { 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 cca2041b8b8..d16dcf54b39 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 @@ -70,9 +70,9 @@ import org.apache.hadoop.util.ToolRunner; public class WALPlayer extends Configured implements Tool { private static final Log LOG = LogFactory.getLog(WALPlayer.class); final static String NAME = "WALPlayer"; - final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output"; - final static String TABLES_KEY = "wal.input.tables"; - final static String TABLE_MAP_KEY = "wal.input.tablesmap"; + public final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output"; + public final static String TABLES_KEY = "wal.input.tables"; + public final static String TABLE_MAP_KEY = "wal.input.tablesmap"; // This relies on Hadoop Configuration to handle warning about deprecated configs and // to set the correct non-deprecated configs when an old one shows up. @@ -84,6 +84,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); } @@ -93,7 +96,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 @@ -105,7 +108,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); } } @@ -148,7 +153,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)) { @@ -159,8 +166,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 { @@ -176,31 +187,41 @@ 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(); } } - /** - * @param cell - * @return Return true if we are to emit this cell. - */ protected boolean filter(Context context, final Cell cell) { return true; } + @Override + protected void + cleanup(Mapper.Context context) + throws IOException, InterruptedException { + super.cleanup(context); + } + @Override 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) { + if (tableMap == null) { + tableMap = tablesToUse; + } + if (tablesToUse == null) { // Then user wants all tables. - } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) { + } else if (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."); + throw new IOException("Incorrect table mapping specified ."); } int i = 0; if (tablesToUse != null) { @@ -214,7 +235,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 @@ -243,7 +266,7 @@ public class WALPlayer extends Configured implements Tool { Configuration conf = getConf(); setupTime(conf, WALInputFormat.START_TIME_KEY); setupTime(conf, WALInputFormat.END_TIME_KEY); - Path inputDir = new Path(args[0]); + String inputDirs = args[0]; String[] tables = args[1].split(","); String[] tableMap; if (args.length > 2) { @@ -257,13 +280,18 @@ public class WALPlayer extends Configured implements Tool { } conf.setStrings(TABLES_KEY, tables); conf.setStrings(TABLE_MAP_KEY, tableMap); - Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir)); + Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + System.currentTimeMillis())); job.setJarByClass(WALPlayer.class); - FileInputFormat.setInputPaths(job, inputDir); + + FileInputFormat.addInputPaths(job, inputDirs); + job.setInputFormatClass(WALInputFormat.class); job.setMapOutputKeyClass(ImmutableBytesWritable.class); + String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); if (hfileOutPath != null) { + LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs); + // the bulk HFile case if (tables.length != 1) { throw new IOException("Exactly one table must be specified for the bulk export option"); @@ -299,7 +327,9 @@ public class WALPlayer extends Configured implements Tool { return job; } - /* + + /** + * Print usage * @param errorMsg Error message. Can be null. */ private void usage(final String errorMsg) { @@ -309,7 +339,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 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."); @@ -322,10 +353,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/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 177ee3251d6..f86f8000e4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -74,10 +74,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegi import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.*; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; @@ -88,9 +86,129 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockH import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; @@ -233,6 +351,7 @@ public class MasterRpcServices extends RSRpcServices /** * @return list of blocking services and their security info classes that this server supports */ + @Override protected List getServices() { List bssi = new ArrayList<>(5); bssi.add(new BlockingServiceAndInterface( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 54b68d39387..55d58e008d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -62,7 +62,7 @@ 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.procedure.ZKProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; @@ -1114,7 +1114,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // setup the default procedure coordinator String name = master.getServerName().toString(); ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, opThreads); - ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs( + ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator( master.getZooKeeper(), SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, name); this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); 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/ZKProcedureCoordinator.java similarity index 98% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java index 4632d2316fa..b656894ee53 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/ZKProcedureCoordinator.java @@ -35,8 +35,8 @@ import org.apache.zookeeper.KeeperException; * ZooKeeper based {@link ProcedureCoordinatorRpcs} for a {@link ProcedureCoordinator} */ @InterfaceAudience.Private -public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { - private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class); +public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { + private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinator.class); private ZKProcedureUtil zkProc = null; protected ProcedureCoordinator coordinator = null; // if started this should be non-null @@ -51,8 +51,8 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { * @param coordName name of the node running the coordinator * @throws KeeperException if an unexpected zk error occurs */ - public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher, - String procedureClass, String coordName) throws KeeperException { + public ZKProcedureCoordinator(ZooKeeperWatcher watcher, + String procedureClass, String coordName) { this.watcher = watcher; this.procedureType = procedureClass; this.coordName = coordName; @@ -179,6 +179,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { * Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about. * @return true if succeed, false if encountered initialization errors. */ + @Override final public boolean start(final ProcedureCoordinator coordinator) { if (this.coordinator != null) { throw new IllegalStateException( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java index 7b624a5d606..30921147469 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java @@ -42,7 +42,7 @@ 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.procedure.ZKProcedureCoordinatorRpcs; +import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; @@ -98,7 +98,7 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager { // setup the procedure coordinator String name = master.getServerName().toString(); ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, threads); - ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs( + ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator( master.getZooKeeper(), getProcedureSignature(), name); this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); 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 be4cca05988..b3b5113e350 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 @@ -199,13 +199,13 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.data.Stat; +import sun.misc.Signal; +import sun.misc.SignalHandler; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import sun.misc.Signal; -import sun.misc.SignalHandler; - /** * HRegionServer makes a set of HRegions available to clients. It checks in with * the HMaster. There are many HRegionServers in a single HBase deployment. @@ -385,7 +385,7 @@ public class HRegionServer extends HasThread implements // WAL roller. log is protected rather than private to avoid // eclipse warning when accessed by inner classes - final LogRoller walRoller; + protected final LogRoller walRoller; // flag set after we're done setting up server threads final AtomicBoolean online = new AtomicBoolean(false); @@ -535,7 +535,6 @@ public class HRegionServer extends HasThread implements // Disable usage of meta replicas in the regionserver this.conf.setBoolean(HConstants.USE_META_REPLICAS, false); - // Config'ed params this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); @@ -640,7 +639,7 @@ public class HRegionServer extends HasThread implements int cleanerInterval = conf.getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000); this.compactedFileDischarger = - new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this); + new CompactedHFilesDischarger(cleanerInterval, this, this); choreService.scheduleChore(compactedFileDischarger); } @@ -859,7 +858,7 @@ public class HRegionServer extends HasThread implements rspmHost.loadProcedures(conf); rspmHost.initialize(this); } catch (KeeperException e) { - this.abort("Failed to reach zk cluster when creating procedure handler.", 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); @@ -1948,6 +1947,10 @@ public class HRegionServer extends HasThread implements return wal; } + public LogRoller getWalRoller() { + return walRoller; + } + @Override public Connection getConnection() { return getClusterConnection(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java index 9dd85d86bd9..bf1493342f7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java @@ -368,6 +368,11 @@ public abstract class AbstractFSWALProvider> implemen return false; } + public static boolean isArchivedLogFile(Path p) { + String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR; + return p.toString().contains(oldLog); + } + /** * Get prefix of the log from its name, assuming WAL name in format of * log_prefix.filenumber.log_suffix diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index a6c7f68a2a4..8a4ed7271d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -49,7 +49,6 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import edu.umd.cs.findbugs.annotations.Nullable; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.logging.Log; @@ -117,6 +116,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.RegionSplitter; +import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; @@ -137,6 +137,8 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +import edu.umd.cs.findbugs.annotations.Nullable; + /** * Facility for testing HBase. Replacement for * old HBaseTestCase and HBaseClusterTestCase functionality. @@ -2171,6 +2173,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } } + public void loadRandomRows(final Table t, final byte[] f, int rowSize, int totalRows) + throws IOException { + Random r = new Random(); + byte[] row = new byte[rowSize]; + for (int i = 0; i < totalRows; i++) { + r.nextBytes(row); + Put put = new Put(row); + put.addColumn(f, new byte[]{0}, new byte[]{0}); + t.put(put); + } + } + public void verifyNumericRows(Table table, final byte[] f, int startRow, int endRow, int replicaId) throws IOException { @@ -3336,6 +3350,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { this.conf.getLong("hbase.client.sync.wait.timeout.msec", 60000)); } + /** + * Waith until all system table's regions get assigned + * @throws IOException + */ + public void waitUntilAllSystemRegionsAssigned() throws IOException { + waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + waitUntilAllRegionsAssigned(TableName.NAMESPACE_TABLE_NAME); + } + /** * Wait until all regions for a table in hbase:meta have a non-empty * info:server, or until timeout. This means all regions have been deployed, @@ -3801,12 +3824,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public static int createPreSplitLoadTestTable(Configuration conf, HTableDescriptor desc, HColumnDescriptor[] hcds, int numRegionsPerServer) throws IOException { + + return createPreSplitLoadTestTable(conf, desc, hcds, + new RegionSplitter.HexStringSplit(), numRegionsPerServer); + } + + /** + * Creates a pre-split table for load testing. If the table already exists, + * logs a warning and continues. + * @return the number of regions the table was split into + */ + public static int createPreSplitLoadTestTable(Configuration conf, + HTableDescriptor desc, HColumnDescriptor[] hcds, + SplitAlgorithm splitter, int numRegionsPerServer) throws IOException { for (HColumnDescriptor hcd : hcds) { if (!desc.hasFamily(hcd.getName())) { desc.addFamily(hcd); } } - int totalNumberOfRegions = 0; Connection unmanagedConnection = ConnectionFactory.createConnection(conf); Admin admin = unmanagedConnection.getAdmin(); @@ -3825,7 +3860,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { "pre-splitting table into " + totalNumberOfRegions + " regions " + "(regions per server: " + numRegionsPerServer + ")"); - byte[][] splits = new RegionSplitter.HexStringSplit().split( + byte[][] splits = splitter.split( totalNumberOfRegions); admin.createTable(desc, splits); 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..ec885497d6f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -0,0 +1,293 @@ +/* + * + * 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.Iterator; +import java.util.List; +import java.util.Map.Entry; + +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.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Durability; +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.client.Table; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALFactory; +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 = TableName.valueOf("table1"); + protected static HTableDescriptor table1Desc; + protected static TableName table2 = TableName.valueOf("table2"); + protected static TableName table3 = TableName.valueOf("table3"); + protected static TableName table4 = TableName.valueOf("table4"); + + protected static TableName table1_restore = TableName.valueOf("ns1:table1_restore"); + protected static TableName table2_restore = TableName.valueOf("ns2:table2_restore"); + protected static TableName table3_restore = TableName.valueOf("ns3:table3_restore"); + protected static TableName table4_restore = TableName.valueOf("ns4:table4_restore"); + + protected static final int NB_ROWS_IN_BATCH = 99; + protected static final byte[] qualName = Bytes.toBytes("q1"); + protected static final byte[] famName = Bytes.toBytes("f"); + + protected static String BACKUP_ROOT_DIR = "/backupUT"; + protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT"; + protected static String provider = "defaultProvider"; + + /** + * @throws java.lang.Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + conf1 = TEST_UTIL.getConfiguration(); + conf1.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf1); + BackupManager.decorateRegionServerConfiguration(conf1); + conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); + // Set MultiWAL (with 2 default WAL files per RS) + conf1.set(WALFactory.WAL_PROVIDER, provider); + TEST_UTIL.startMiniZKCluster(); + MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster(); + + 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); + createTables(); + populateFromMasterConfig(TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(), conf1); + } + + private static void populateFromMasterConfig(Configuration masterConf, Configuration conf) { + Iterator> it = masterConf.iterator(); + while (it.hasNext()) { + Entry e = it.next(); + conf.set(e.getKey(), e.getValue()); + } + } + + /** + * @throws java.lang.Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin()); + SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); + TEST_UTIL2.shutdownMiniCluster(); + TEST_UTIL.shutdownMiniCluster(); + TEST_UTIL.shutdownMiniMapReduceCluster(); + } + + HTable insertIntoTable(Connection conn, TableName table, byte[] family, int id, int numRows) + throws IOException { + HTable t = (HTable) conn.getTable(table); + Put p1; + for (int i = 0; i < numRows; i++) { + p1 = new Put(Bytes.toBytes("row-" + table + "-" + id + "-" + i)); + p1.addColumn(family, qualName, Bytes.toBytes("val" + i)); + t.put(p1); + } + return t; + } + + + protected BackupRequest createBackupRequest(BackupType type, + List tables, String path) { + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = builder.withBackupType(type) + .withTableList(tables) + .withTargetRootDir(path).build(); + return request; + } + + protected String backupTables(BackupType type, List tables, String path) + throws IOException { + Connection conn = null; + BackupAdmin badmin = null; + String backupId; + try { + conn = ConnectionFactory.createConnection(conf1); + badmin = new BackupAdminImpl(conn); + BackupRequest request = createBackupRequest(type, tables, path); + backupId = badmin.backupTables(request); + } finally { + if (badmin != null) { + badmin.close(); + } + if (conn != null) { + conn.close(); + } + } + return backupId; + } + + protected String fullTableBackup(List tables) throws IOException { + return backupTables(BackupType.FULL, tables, BACKUP_ROOT_DIR); + } + + protected String incrementalTableBackup(List tables) throws IOException { + return backupTables(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + } + + protected static void loadTable(Table table) throws Exception { + + Put p; // 100 + 1 row to t1_syncup + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p = new Put(Bytes.toBytes("row" + i)); + p.setDurability(Durability.SKIP_WAL); + p.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + table.put(p); + } + } + + protected static void createTables() throws Exception { + + long tid = System.currentTimeMillis(); + table1 = TableName.valueOf("ns1:test-" + tid); + HBaseAdmin ha = TEST_UTIL.getHBaseAdmin(); + + // Create namespaces + NamespaceDescriptor desc1 = NamespaceDescriptor.create("ns1").build(); + NamespaceDescriptor desc2 = NamespaceDescriptor.create("ns2").build(); + NamespaceDescriptor desc3 = NamespaceDescriptor.create("ns3").build(); + NamespaceDescriptor desc4 = NamespaceDescriptor.create("ns4").build(); + + ha.createNamespace(desc1); + ha.createNamespace(desc2); + ha.createNamespace(desc3); + ha.createNamespace(desc4); + + HTableDescriptor desc = new HTableDescriptor(table1); + HColumnDescriptor fam = new HColumnDescriptor(famName); + desc.addFamily(fam); + ha.createTable(desc); + table1Desc = desc; + Connection conn = ConnectionFactory.createConnection(conf1); + Table table = conn.getTable(table1); + loadTable(table); + table.close(); + table2 = TableName.valueOf("ns2:test-" + tid + 1); + desc = new HTableDescriptor(table2); + desc.addFamily(fam); + ha.createTable(desc); + table = conn.getTable(table2); + loadTable(table); + table.close(); + table3 = TableName.valueOf("ns3:test-" + tid + 2); + table = TEST_UTIL.createTable(table3, famName); + table.close(); + table4 = TableName.valueOf("ns4:test-" + tid + 3); + table = TEST_UTIL.createTable(table4, famName); + table.close(); + ha.close(); + conn.close(); + } + + protected boolean checkSucceeded(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + if (status == null) return false; + return status.getState() == BackupState.COMPLETE; + } + + protected boolean checkFailed(String backupId) throws IOException { + BackupInfo status = getBackupInfo(backupId); + if (status == null) return false; + return status.getState() == BackupState.FAILED; + } + + private BackupInfo getBackupInfo(String backupId) throws IOException { + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + BackupInfo status = table.readBackupInfo(backupId); + return status; + } + } + + protected BackupAdmin getBackupAdmin() throws IOException { + return new BackupAdminImpl(TEST_UTIL.getConnection()); + } + + /** + * Helper method + */ + protected List toList(String... args) { + List ret = new ArrayList<>(); + for (int i = 0; i < args.length; i++) { + ret.add(TableName.valueOf(args[i])); + } + return ret; + } + + protected void dumpBackupDir() throws IOException { + // Dump Backup Dir + FileSystem fs = FileSystem.get(conf1); + RemoteIterator it = fs.listFiles(new Path(BACKUP_ROOT_DIR), true); + while (it.hasNext()) { + LOG.debug(it.next().getPath()); + } + + } +} 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..280314becbb --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@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"); + List tables = Lists.newArrayList(table3); + LOG.info("Finished Backup " + fullTableBackup(tables)); + } + + /** + * 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"); + + List tables = Lists.newArrayList(table3, table4); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on a single table that does not exist. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullBackupSingleDNE() throws Exception { + + LOG.info("test full backup fails on a single table that does not exist"); + List tables = toList("tabledne"); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on multiple tables that do not exist. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullBackupMultipleDNE() throws Exception { + + LOG.info("test full backup fails on multiple tables that do not exist"); + List tables = toList("table1dne", "table2dne"); + fullTableBackup(tables); + } + + /** + * Verify that full backup fails on tableset containing real and fake tables. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullBackupMixExistAndDNE() throws Exception { + LOG.info("create full backup fails on tableset containing real and fake table"); + + List tables = toList(table1.getNameAsString(), "tabledne"); + fullTableBackup(tables); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java new file mode 100644 index 00000000000..08002fbe0fd --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java @@ -0,0 +1,431 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBackupCommandLineTool { + + private final static String USAGE_DESCRIBE = "Usage: hbase backup describe "; + private final static String USAGE_CREATE = "Usage: hbase backup create"; + private final static String USAGE_HISTORY = "Usage: hbase backup history"; + private final static String USAGE_BACKUP = "Usage: hbase backup"; + private final static String USAGE_DELETE = "Usage: hbase backup delete"; + private final static String USAGE_PROGRESS = "Usage: hbase backup progress"; + private final static String USAGE_SET = "Usage: hbase backup set"; + private final static String USAGE_RESTORE = "Usage: hbase restore"; + + Configuration conf; + + @Before + public void setUpBefore() throws Exception { + conf = HBaseConfiguration.create(); + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + } + + @Test + public void testBackupDriverDescribeHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "describe", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0); + } + + @Test + public void testBackupDriverCreateHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + } + + @Test + public void testBackupDriverHistoryHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "history", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "history", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + } + + @Test + public void testBackupDriverDeleteHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "delete", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + } + + @Test + public void testBackupDriverProgressHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "progress", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "progress", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_PROGRESS) >= 0); + } + + @Test + public void testBackupDriverSetHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "set", "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set", "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_SET) >= 0); + + } + + @Test + public void testBackupDriverHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-help" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "-h" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + } + + @Test + public void testRestoreDriverHelp() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-help" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "-h" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0); + + } + + @Test + public void testBackupDriverUnrecognizedCommand() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "command" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "command" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + } + + @Test + public void testBackupDriverUnrecognizedOption() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "describe", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "history", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "delete", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "set", "-xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_BACKUP) >= 0); + } + + @Test + public void testRestoreDriverUnrecognizedOption() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "-xx" }; + ToolRunner.run(conf, new RestoreDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_RESTORE) >= 0); + + } + + @Test + public void testBackupDriverCreateWrongArgNumber() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "22" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + args = new String[] { "create", "22", "22", "22", "22", "22" }; + ToolRunner.run(conf, new BackupDriver(), args); + + output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_CREATE) >= 0); + } + + @Test + public void testBackupDriverDeleteWrongArgNumber() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "delete" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_DELETE) >= 0); + + } + + @Test + public void testBackupDriverHistoryWrongArgs() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "history", "-n", "xx" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf(USAGE_HISTORY) >= 0); + + } + + @Test + public void testBackupDriverWrongBackupDestination() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "full", "clicks" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf("ERROR: invalid backup destination") >= 0); + + } + + @Test + public void testBackupDriverBackupSetAndList() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + String[] args = new String[] { "create", "full", "file:/", "-t", "clicks", "-s", "s" }; + ToolRunner.run(conf, new BackupDriver(), args); + + String output = baos.toString(); + System.out.println(baos.toString()); + assertTrue(output.indexOf("ERROR: You can specify either backup set or list") >= 0); + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java new file mode 100644 index 00000000000..38724fb783d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDelete extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupDelete.class); + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected + * @throws Exception + */ + @Test + public void testBackupDelete() throws Exception { + LOG.info("test backup delete on a single table with data"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + String[] backupIds = new String[] { backupId }; + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo info = table.readBackupInfo(backupId); + Path path = new Path(info.getBackupRootDir(), backupId); + FileSystem fs = FileSystem.get(path.toUri(), conf1); + assertTrue(fs.exists(path)); + int deleted = getBackupAdmin().deleteBackups(backupIds); + + assertTrue(!fs.exists(path)); + assertTrue(fs.exists(new Path(info.getBackupRootDir()))); + assertTrue(1 == deleted); + table.close(); + LOG.info("delete_backup"); + } + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected + * @throws Exception + */ + @Test + public void testBackupDeleteCommand() throws Exception { + LOG.info("test backup delete on a single table with data: command-line"); + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "delete", backupId }; + // Run backup + + try { + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + } catch (Exception e) { + LOG.error("failed", e); + } + LOG.info("delete_backup"); + String output = baos.toString(); + LOG.info(baos.toString()); + assertTrue(output.indexOf("Deleted 1 backups") >= 0); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java new file mode 100644 index 00000000000..09218710e91 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable + * law or agreed to in writing, software distributed under the License is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License + * for the specific language governing permissions and limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(MediumTests.class) +public class TestBackupDeleteRestore extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupDeleteRestore.class); + + /** + * Verify that load data- backup - delete some data - restore works as expected - deleted data get + * restored. + * @throws Exception + */ + @Test + public void testBackupDeleteRestore() throws Exception { + + LOG.info("test full restore on a single table empty table"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + int numRows = TEST_UTIL.countRows(table1); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + // delete row + try (Table table = TEST_UTIL.getConnection().getTable(table1);) { + Delete delete = new Delete("row0".getBytes()); + table.delete(delete); + hba.flush(table1); + } + + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = null;// new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + tableset, tablemap, true)); + + int numRowsAfterRestore = TEST_UTIL.countRows(table1); + assertEquals(numRows, numRowsAfterRestore); + hba.close(); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java new file mode 100644 index 00000000000..38aa30dc20f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupCommands; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDescribe extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupDescribe.class); + + /** + * Verify that describe works as expected if incorrect backup Id is supplied + * @throws Exception + */ + @Test + public void testBackupDescribe() throws Exception { + + LOG.info("test backup describe on a single table with data"); + + String[] args = new String[] { "describe", "backup_2" }; + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret < 0); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setErr(new PrintStream(baos)); + args = new String[] { "progress" }; + ToolRunner.run(TEST_UTIL.getConfiguration(), new BackupDriver(), args); + + String output = baos.toString(); + LOG.info("Output from progress: " + output); + assertTrue(output.indexOf(BackupCommands.NO_ACTIVE_SESSION_FOUND) >= 0); + } + + @Test + public void testBackupSetCommandWithNonExistentTable() throws Exception { + String[] args = new String[] { "set", "add", "some_set", "table" }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertNotEquals(ret, 0); + } + + @Test + public void testBackupDescribeCommand() throws Exception { + + LOG.info("test backup describe on a single table with data: command-line"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + BackupInfo info = getBackupAdmin().getBackupInfo(backupId); + assertTrue(info.getState() == BackupState.COMPLETE); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "describe", backupId }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + String response = baos.toString(); + assertTrue(response.indexOf(backupId) > 0); + assertTrue(response.indexOf("COMPLETE") > 0); + + BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); + BackupInfo status = table.readBackupInfo(backupId); + String desc = status.getShortDescription(); + table.close(); + assertTrue(response.indexOf(desc) >= 0); + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java new file mode 100644 index 00000000000..d173e200b4e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +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.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +/** + * Create multiple backups for two tables: table1, table2 then perform 1 delete + */ +@Category(LargeTests.class) +public class TestBackupMultipleDeletes extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(TestBackupMultipleDeletes.class); + + @Test + public void testBackupMultipleDeletes() throws Exception { + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + List tables = Lists.newArrayList(table1, table2); + HBaseAdmin admin = null; + Connection conn = ConnectionFactory.createConnection(conf1); + admin = (HBaseAdmin) conn.getAdmin(); + BackupAdmin client = new BackupAdminImpl(conn); + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull)); + // #2 - insert some data to table table1 + 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.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2); + t1.close(); + // #3 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc1 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc1)); + // #4 - insert some data to table table2 + HTable t2 = (HTable) conn.getTable(table2); + Put p2 = null; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p2 = new Put(Bytes.toBytes("row-t2" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + // #5 - incremental backup for table1, table2 + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc2)); + // #6 - insert some data to table table1 + t1 = (HTable) conn.getTable(table1); + for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + // #7 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc3 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc3)); + // #8 - insert some data to table table2 + t2 = (HTable) conn.getTable(table2); + for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) { + p2 = new Put(Bytes.toBytes("row-t1" + i)); + p2.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + t2.put(p2); + } + // #9 - incremental backup for table1, table2 + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc4 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc4)); + // #10 full backup for table3 + tables = Lists.newArrayList(table3); + request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdFull2)); + // #11 - incremental backup for table3 + tables = Lists.newArrayList(table3); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdInc5 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdInc5)); + LOG.error("Delete backupIdInc2"); + client.deleteBackups(new String[] { backupIdInc2 }); + LOG.error("Delete backupIdInc2 done"); + List list = client.getHistory(100); + // First check number of backup images before and after + assertEquals(4, list.size()); + // then verify that no backupIdInc2,3,4 + Set ids = new HashSet(); + ids.add(backupIdInc2); + ids.add(backupIdInc3); + ids.add(backupIdInc4); + for (BackupInfo info : list) { + String backupId = info.getBackupId(); + if (ids.contains(backupId)) { + assertTrue(false); + } + } + // Verify that backupInc5 contains only table3 + boolean found = false; + for (BackupInfo info : list) { + String backupId = info.getBackupId(); + if (backupId.equals(backupIdInc5)) { + assertTrue(info.getTables().size() == 1); + assertEquals(table3, info.getTableNames().get(0)); + found = true; + } + } + assertTrue(found); + admin.close(); + conn.close(); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java new file mode 100644 index 00000000000..740c396aeee --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +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.backup.util.BackupUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupShowHistory extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupShowHistory.class); + + private boolean findBackup(List history, String backupId) { + assertTrue(history.size() > 0); + boolean success = false; + for (BackupInfo info : history) { + if (info.getBackupId().equals(backupId)) { + success = true; + break; + } + } + return success; + } + + /** + * Verify that full backup is created on a single table with data correctly. Verify that history + * works as expected + * @throws Exception + */ + @Test + public void testBackupHistory() throws Exception { + + LOG.info("test backup history on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + + List history = getBackupAdmin().getHistory(10); + assertTrue(findBackup(history, backupId)); + BackupInfo.Filter nullFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + return true; + } + }; + history = BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), nullFilter); + assertTrue(findBackup(history, backupId)); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + LOG.info("show_history"); + String output = baos.toString(); + LOG.info(output); + baos.close(); + assertTrue(output.indexOf(backupId) > 0); + + tableList = Lists.newArrayList(table2); + String backupId2 = fullTableBackup(tableList); + assertTrue(checkSucceeded(backupId2)); + LOG.info("backup complete: " + table2); + BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo image) { + if (table1 == null) return true; + List names = image.getTableNames(); + return names.contains(table1); + } + }; + BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() { + @Override + public boolean apply(BackupInfo info) { + String backupId = info.getBackupId(); + return backupId.startsWith("backup"); + } + }; + + history = getBackupAdmin().getHistory(10, tableNameFilter, tableSetFilter); + assertTrue(history.size() > 0); + boolean success = true; + for (BackupInfo info : history) { + if (!info.getTableNames().contains(table1)) { + success = false; + break; + } + } + assertTrue(success); + + history = + BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), tableNameFilter, + tableSetFilter); + assertTrue(history.size() > 0); + success = true; + for (BackupInfo info : history) { + if (!info.getTableNames().contains(table1)) { + success = false; + break; + } + } + assertTrue(success); + + args = + new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR, + "-t", "table1", "-s", "backup" }; + // Run backup + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + LOG.info("show_history"); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java new file mode 100644 index 00000000000..a904d9b6f14 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupStatusProgress extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestBackupStatusProgress.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testBackupStatusProgress() throws Exception { + + LOG.info("test backup status/progress on a single table with data"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + + BackupInfo info = getBackupAdmin().getBackupInfo(backupId); + assertTrue(info.getState() == BackupState.COMPLETE); + + LOG.debug(info.getShortDescription()); + assertTrue(info.getProgress() > 0); + + } + + @Test + public void testBackupStatusProgressCommand() throws Exception { + + LOG.info("test backup status/progress on a single table with data: command-line"); + + List tableList = Lists.newArrayList(table1); + String backupId = fullTableBackup(tableList); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + String[] args = new String[] { "describe", backupId }; + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + String responce = baos.toString(); + assertTrue(responce.indexOf(backupId) > 0); + assertTrue(responce.indexOf("COMPLETE") > 0); + + baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + args = new String[] { "progress", backupId }; + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + responce = baos.toString(); + assertTrue(responce.indexOf(backupId) >= 0); + assertTrue(responce.indexOf("progress") > 0); + assertTrue(responce.indexOf("100") > 0); + + } +} diff --git a/hbase-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..5814d87aa0d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.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 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.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test cases for backup system table API + */ +@Category(MediumTests.class) +public class TestBackupSystemTable { + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + protected static Configuration conf = UTIL.getConfiguration(); + protected static MiniHBaseCluster cluster; + protected static Connection conn; + protected BackupSystemTable table; + + @BeforeClass + public static void setUp() throws Exception { + conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true); + BackupManager.decorateMasterConfiguration(conf); + BackupManager.decorateRegionServerConfiguration(conf); + cluster = UTIL.startMiniCluster(); + conn = UTIL.getConnection(); + } + + @Before + public void before() throws IOException { + table = new BackupSystemTable(conn); + } + + @After + public void after() { + if (table != null) { + table.close(); + } + + } + + @Test + public void testUpdateReadDeleteBackupStatus() throws IOException { + BackupInfo ctx = createBackupInfo(); + table.updateBackupInfo(ctx); + BackupInfo readCtx = table.readBackupInfo(ctx.getBackupId()); + assertTrue(compare(ctx, readCtx)); + // try fake backup id + readCtx = table.readBackupInfo("fake"); + assertNull(readCtx); + // delete backup info + table.deleteBackupInfo(ctx.getBackupId()); + readCtx = table.readBackupInfo(ctx.getBackupId()); + assertNull(readCtx); + cleanBackupTable(); + } + + @Test + public void testWriteReadBackupStartCode() throws IOException { + Long code = 100L; + table.writeBackupStartCode(code, "root"); + String readCode = table.readBackupStartCode("root"); + assertEquals(code, new Long(Long.parseLong(readCode))); + cleanBackupTable(); + } + + private void cleanBackupTable() throws IOException { + Admin admin = UTIL.getHBaseAdmin(); + admin.disableTable(BackupSystemTable.getTableName(conf)); + admin.truncateTable(BackupSystemTable.getTableName(conf), true); + if (admin.isTableDisabled(BackupSystemTable.getTableName(conf))) { + admin.enableTable(BackupSystemTable.getTableName(conf)); + } + } + + @Test + public void testBackupHistory() throws IOException { + int n = 10; + List list = createBackupInfoList(n); + + // Load data + for (BackupInfo bc : list) { + // Make sure we set right status + bc.setState(BackupState.COMPLETE); + table.updateBackupInfo(bc); + } + + // Reverse list for comparison + Collections.reverse(list); + List history = table.getBackupHistory(); + assertTrue(history.size() == n); + + for (int i = 0; i < n; i++) { + BackupInfo ctx = list.get(i); + BackupInfo data = history.get(i); + assertTrue(compare(ctx, data)); + } + + cleanBackupTable(); + + } + + @Test + public void testBackupDelete() throws IOException { + + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + int n = 10; + List list = createBackupInfoList(n); + + // Load data + for (BackupInfo bc : list) { + // Make sure we set right status + bc.setState(BackupState.COMPLETE); + table.updateBackupInfo(bc); + } + + // Verify exists + for (BackupInfo bc : list) { + assertNotNull(table.readBackupInfo(bc.getBackupId())); + } + + // Delete all + for (BackupInfo bc : list) { + table.deleteBackupInfo(bc.getBackupId()); + } + + // Verify do not exists + for (BackupInfo bc : list) { + assertNull(table.readBackupInfo(bc.getBackupId())); + } + + cleanBackupTable(); + } + + } + + @Test + public void testRegionServerLastLogRollResults() throws IOException { + String[] servers = new String[] { "server1", "server2", "server3" }; + Long[] timestamps = new Long[] { 100L, 102L, 107L }; + + for (int i = 0; i < servers.length; i++) { + table.writeRegionServerLastLogRollResult(servers[i], timestamps[i], "root"); + } + + HashMap result = table.readRegionServerLastLogRollResult("root"); + assertTrue(servers.length == result.size()); + Set keys = result.keySet(); + String[] keysAsArray = new String[keys.size()]; + keys.toArray(keysAsArray); + Arrays.sort(keysAsArray); + + for (int i = 0; i < keysAsArray.length; i++) { + assertEquals(keysAsArray[i], servers[i]); + Long ts1 = timestamps[i]; + Long ts2 = result.get(keysAsArray[i]); + assertEquals(ts1, ts2); + } + + cleanBackupTable(); + } + + @Test + public void testIncrementalBackupTableSet() throws IOException { + TreeSet tables1 = new TreeSet<>(); + + tables1.add(TableName.valueOf("t1")); + tables1.add(TableName.valueOf("t2")); + tables1.add(TableName.valueOf("t3")); + + TreeSet tables2 = new TreeSet<>(); + + tables2.add(TableName.valueOf("t3")); + tables2.add(TableName.valueOf("t4")); + tables2.add(TableName.valueOf("t5")); + + table.addIncrementalBackupTableSet(tables1, "root"); + BackupSystemTable table = new BackupSystemTable(conn); + TreeSet res1 = (TreeSet) table.getIncrementalBackupTableSet("root"); + assertTrue(tables1.size() == res1.size()); + Iterator desc1 = tables1.descendingIterator(); + Iterator desc2 = res1.descendingIterator(); + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + + table.addIncrementalBackupTableSet(tables2, "root"); + TreeSet res2 = (TreeSet) table.getIncrementalBackupTableSet("root"); + assertTrue((tables2.size() + tables1.size() - 1) == res2.size()); + + tables1.addAll(tables2); + + desc1 = tables1.descendingIterator(); + desc2 = res2.descendingIterator(); + + while (desc1.hasNext()) { + assertEquals(desc1.next(), desc2.next()); + } + cleanBackupTable(); + + } + + @Test + public void testRegionServerLogTimestampMap() throws IOException { + TreeSet tables = new TreeSet<>(); + + tables.add(TableName.valueOf("t1")); + tables.add(TableName.valueOf("t2")); + tables.add(TableName.valueOf("t3")); + + HashMap rsTimestampMap = new HashMap(); + + rsTimestampMap.put("rs1:100", 100L); + rsTimestampMap.put("rs2:100", 101L); + rsTimestampMap.put("rs3:100", 103L); + + table.writeRegionServerLogTimestamp(tables, rsTimestampMap, "root"); + + HashMap> result = table.readLogTimestampMap("root"); + + assertTrue(tables.size() == result.size()); + + for (TableName t : tables) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1:100"), new Long(100L)); + assertEquals(rstm.get("rs2:100"), new Long(101L)); + assertEquals(rstm.get("rs3:100"), new Long(103L)); + } + + Set tables1 = new TreeSet<>(); + + tables1.add(TableName.valueOf("t3")); + tables1.add(TableName.valueOf("t4")); + tables1.add(TableName.valueOf("t5")); + + HashMap rsTimestampMap1 = new HashMap(); + + rsTimestampMap1.put("rs1:100", 200L); + rsTimestampMap1.put("rs2:100", 201L); + rsTimestampMap1.put("rs3:100", 203L); + + table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1, "root"); + + result = table.readLogTimestampMap("root"); + + assertTrue(5 == result.size()); + + for (TableName t : tables) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + if (t.equals(TableName.valueOf("t3")) == false) { + assertEquals(rstm.get("rs1:100"), new Long(100L)); + assertEquals(rstm.get("rs2:100"), new Long(101L)); + assertEquals(rstm.get("rs3:100"), new Long(103L)); + } else { + assertEquals(rstm.get("rs1:100"), new Long(200L)); + assertEquals(rstm.get("rs2:100"), new Long(201L)); + assertEquals(rstm.get("rs3:100"), new Long(203L)); + } + } + + for (TableName t : tables1) { + HashMap rstm = result.get(t); + assertNotNull(rstm); + assertEquals(rstm.get("rs1:100"), new Long(200L)); + assertEquals(rstm.get("rs2:100"), new Long(201L)); + assertEquals(rstm.get("rs3:100"), new Long(203L)); + } + + cleanBackupTable(); + + } + + @Test + public void testAddWALFiles() throws IOException { + 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", "root"); + + assertTrue(table.isWALFileDeletable(files.get(0))); + assertTrue(table.isWALFileDeletable(files.get(1))); + assertTrue(table.isWALFileDeletable(files.get(2))); + assertFalse(table.isWALFileDeletable(newFile)); + + cleanBackupTable(); + } + + /** + * Backup set tests + */ + + @Test + public void testBackupSetAddNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals(tables[i])); + } + cleanBackupTable(); + } + + } + + @Test + public void testBackupSetAddExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] addTables = new String[] { "table4", "table5", "table6" }; + table.addToBackupSet(setName, addTables); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length + addTables.length); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetAddExistsIntersects() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] addTables = new String[] { "table3", "table4", "table5", "table6" }; + table.addToBackupSet(setName, addTables); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length + addTables.length - 1); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemoveSomeNotExists() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] removeTables = new String[] { "table4", "table5", "table6" }; + table.removeFromBackupSet(setName, removeTables); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length - 1); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetRemove() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + String[] removeTables = new String[] { "table4", "table3" }; + table.removeFromBackupSet(setName, removeTables); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames != null); + assertTrue(tnames.size() == tables.length - 2); + for (int i = 0; i < tnames.size(); i++) { + assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1))); + } + cleanBackupTable(); + } + } + + @Test + public void testBackupSetDelete() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName = "name"; + table.addToBackupSet(setName, tables); + table.deleteBackupSet(setName); + + List tnames = table.describeBackupSet(setName); + assertTrue(tnames == null); + cleanBackupTable(); + } + } + + @Test + public void testBackupSetList() throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + + String[] tables = new String[] { "table1", "table2", "table3", "table4" }; + String setName1 = "name1"; + String setName2 = "name2"; + table.addToBackupSet(setName1, tables); + table.addToBackupSet(setName2, tables); + + List list = table.listBackupSets(); + + assertTrue(list.size() == 2); + assertTrue(list.get(0).equals(setName1)); + assertTrue(list.get(1).equals(setName2)); + + cleanBackupTable(); + } + } + + private boolean compare(BackupInfo one, BackupInfo two) { + return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType()) + && one.getBackupRootDir().equals(two.getBackupRootDir()) + && one.getStartTs() == two.getStartTs() && one.getCompleteTs() == two.getCompleteTs(); + } + + private BackupInfo createBackupInfo() { + + BackupInfo ctxt = + new BackupInfo("backup_" + System.nanoTime(), BackupType.FULL, new TableName[] { + TableName.valueOf("t1"), TableName.valueOf("t2"), TableName.valueOf("t3") }, + "/hbase/backup"); + ctxt.setStartTs(System.currentTimeMillis()); + ctxt.setCompleteTs(System.currentTimeMillis() + 1); + return ctxt; + } + + private List createBackupInfoList(int size) { + List list = new ArrayList(); + for (int i = 0; i < size; i++) { + list.add(createBackupInfo()); + 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..124d19f026a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +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); + + @Test + public void testFullBackupMultipleCommand() throws Exception { + LOG.info("test full backup on a multiple tables with data: command-line"); + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + String[] args = + new String[] { "create", "full", BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() + "," + table2.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + int after = table.getBackupHistory().size(); + assertTrue(after == before + 1); + for (BackupInfo data : backups) { + String backupId = data.getBackupId(); + assertTrue(checkSucceeded(backupId)); + } + } + LOG.info("backup complete"); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java new file mode 100644 index 00000000000..4dc894b6edc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestFullBackupSet extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestFullBackupSet.class); + + /** + * Verify that full backup is created on a single table with data correctly. + * @throws Exception + */ + @Test + public void testFullBackupSetExist() throws Exception { + + LOG.info("Test full backup, backup set exists"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + assertTrue(backups.size() == 1); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + // Restore from set into other table + args = + new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m", + table1_restore.getNameAsString(), "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + // Verify number of rows in both tables + assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + LOG.info("restore into other table is complete"); + hba.close(); + + } + + } + + @Test + public void testFullBackupSetDoesNotExist() throws Exception { + + LOG.info("test full backup, backup set does not exist"); + String name = "name1"; + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret != 0); + + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java new file mode 100644 index 00000000000..6b007f97efd --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestFullBackupSetRestoreSet extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestFullBackupSetRestoreSet.class); + + @Test + public void testFullRestoreSetToOtherTable() throws Exception { + + LOG.info("Test full restore set"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + assertTrue(backups.size() == 1); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + // Restore from set into other table + args = + new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m", + table1_restore.getNameAsString(), "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + // Verify number of rows in both tables + assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + LOG.info("restore into other table is complete"); + hba.close(); + } + } + + @Test + public void testFullRestoreSetToSameTable() throws Exception { + + LOG.info("Test full restore set to same table"); + + // Create set + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + String name = "name1"; + table.addToBackupSet(name, new String[] { table1.getNameAsString() }); + List names = table.describeBackupSet(name); + + assertNotNull(names); + assertTrue(names.size() == 1); + assertTrue(names.get(0).equals(table1)); + + String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name }; + // Run backup + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + List backups = table.getBackupHistory(); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + int count = TEST_UTIL.countRows(table1); + TEST_UTIL.deleteTable(table1); + + // Restore from set into other table + args = new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-o" }; + // Run backup + ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1)); + // Verify number of rows in both tables + assertEquals(count, TEST_UTIL.countRows(table1)); + LOG.info("restore into same table is complete"); + hba.close(); + + } + + } + +} \ 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..61bad6891a4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java @@ -0,0 +1,345 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable + * law or agreed to in writing, software distributed under the License is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License + * for the specific language governing permissions and limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@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"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + tableset, tablemap, false)); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + hba.close(); + } + + + + @Test + public void testFullRestoreSingleCommand() throws Exception { + + LOG.info("test full restore on a single table empty table: command-line"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m", + table1_restore.getNameAsString() }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(table1_restore); + hba.close(); + } + + @Test + public void testFullRestoreCheckCommand() throws Exception { + + LOG.info("test full restore on a single table: command-line, check only"); + + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + LOG.info("backup complete"); + assertTrue(checkSucceeded(backupId)); + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m", + table1_restore.getNameAsString(), "-c" }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + //Verify that table has not been restored + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertFalse(hba.tableExists(table1_restore)); + } + + /** + * 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"); + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, tablemap, false)); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } + + /** + * Verify that multiple tables are restored to new tables. + * @throws Exception + */ + @Test + public void testFullRestoreMultipleCommand() throws Exception { + LOG.info("create full backup image on multiple tables: command-line"); + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", StringUtils.join(restore_tableset, ","), + "-m", StringUtils.join(tablemap, ",") }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } + + /** + * Verify that a single table is restored using overwrite + * @throws Exception + */ + @Test + public void testFullRestoreSingleOverwrite() throws Exception { + + LOG.info("test full restore on a single table empty table"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { table1 }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + tableset, null, true)); + } + + /** + * Verify that a single table is restored using overwrite + * @throws Exception + */ + @Test + public void testFullRestoreSingleOverwriteCommand() throws Exception { + + LOG.info("test full restore on a single table empty table: command-line"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", StringUtils.join(tableset, ","), "-o" }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret == 0); + + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1)); + hba.close(); + + } + + /** + * 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"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, null, true)); + } + + /** + * Verify that multiple tables are restored to new tables using overwrite. + * @throws Exception + */ + @Test + public void testFullRestoreMultipleOverwriteCommand() throws Exception { + LOG.info("create full backup image on multiple tables: command-line"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = new TableName[] { table2, table3 }; + // restore [tableMapping] + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, "-t", + StringUtils.join(restore_tableset, ","), "-o" }; + // Run backup + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + + assertTrue(ret == 0); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table2)); + assertTrue(hba.tableExists(table3)); + hba.close(); + } + + /** + * 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"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { TableName.valueOf("faketable") }; + TableName[] tablemap = new TableName[] { table1_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + tableset, tablemap, false)); + } + + /** + * Verify that restore fails on a single table that does not exist. + * @throws Exception + */ + @Test + public void testFullRestoreSingleDNECommand() throws Exception { + + LOG.info("test restore fails on a single table that does not exist: command-line"); + List tables = Lists.newArrayList(table1); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete"); + + TableName[] tableset = new TableName[] { TableName.valueOf("faketable") }; + TableName[] tablemap = new TableName[] { table1_restore }; + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, StringUtils.join(tableset, ","), "-m", + StringUtils.join(tablemap, ",") }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret != 0); + + } + + /** + * Verify that restore fails on multiple tables that do not exist. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testFullRestoreMultipleDNE() throws Exception { + + LOG.info("test restore fails on multiple tables that do not exist"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = + new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, + restore_tableset, tablemap, false)); + } + + /** + * Verify that restore fails on multiple tables that do not exist. + * @throws Exception + */ + @Test + public void testFullRestoreMultipleDNECommand() throws Exception { + + LOG.info("test restore fails on multiple tables that do not exist: command-line"); + + List tables = Lists.newArrayList(table2, table3); + String backupId = fullTableBackup(tables); + assertTrue(checkSucceeded(backupId)); + + TableName[] restore_tableset = + new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + String[] args = + new String[] { BACKUP_ROOT_DIR, backupId, StringUtils.join(restore_tableset, ","), "-m", + StringUtils.join(tablemap, ",") }; + // Run restore + int ret = ToolRunner.run(conf1, new RestoreDriver(), args); + assertTrue(ret != 0); + } +} \ 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..77bed436fb3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -0,0 +1,200 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.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.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +@RunWith(Parameterized.class) +public class TestIncrementalBackup extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class); + + @Parameterized.Parameters + public static Collection data() { + provider = "multiwal"; + List params = new ArrayList(); + params.add(new Object[] { Boolean.TRUE }); + return params; + } + + public TestIncrementalBackup(Boolean b) { + } + + // implement all test cases in 1 test since incremental backup/restore has dependencies + @Test + public void TestIncBackupRestore() throws Exception { + + int ADD_ROWS = 99; + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + final byte[] fam3Name = Bytes.toBytes("f3"); + table1Desc.addFamily(new HColumnDescriptor(fam3Name)); + HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); + + Connection conn = ConnectionFactory.createConnection(conf1); + int NB_ROWS_FAM3 = 6; + insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close(); + + HBaseAdmin admin = null; + admin = (HBaseAdmin) conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table + HTable t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); + LOG.debug("writing " + ADD_ROWS + " rows to " + table1); + + Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3); + t1.close(); + LOG.debug("written " + ADD_ROWS + " rows to " + table1); + + 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.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + 5); + t2.close(); + LOG.debug("written " + 5 + " rows to " + table2); + // split table1 + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + List regions = cluster.getRegions(table1); + + byte[] name = regions.get(0).getRegionInfo().getRegionName(); + long startSplitTime = EnvironmentEdgeManager.currentTime(); + admin.splitRegion(name); + + while (!admin.isTableAvailable(table1)) { + Thread.sleep(100); + } + + long endSplitTime = EnvironmentEdgeManager.currentTime(); + + // split finished + LOG.debug("split finished in =" + (endSplitTime - startSplitTime)); + + // #3 - incremental backup for multiple tables + tables = Lists.newArrayList(table1, table2); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple)); + + // add column family f2 to table1 + final byte[] fam2Name = Bytes.toBytes("f2"); + table1Desc.addFamily(new HColumnDescriptor(fam2Name)); + // drop column family f3 + table1Desc.removeFamily(fam3Name); + HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); + + int NB_ROWS_FAM2 = 7; + HTable t3 = insertIntoTable(conn, table1, fam2Name, 2, NB_ROWS_FAM2); + t3.close(); + + // #3 - incremental backup for multiple tables + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple2 = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple2)); + + // #4 - restore full backup for all tables, without overwrite + TableName[] tablesRestoreFull = new TableName[] { table1, table2 }; + + TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore }; + + LOG.debug("Restoring full " + backupIdFull); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, + tablesRestoreFull, tablesMapFull, false)); + + // #5.1 - check tables for full restore + HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + assertTrue(hAdmin.tableExists(table2_restore)); + + hAdmin.close(); + + // #5.2 - checking row count of tables for full restore + HTable hTable = (HTable) conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + NB_ROWS_FAM3); + hTable.close(); + + hTable = (HTable) conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH); + hTable.close(); + + // #6 - restore incremental backup for multiple tables, with overwrite + TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2, + false, tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + + hTable = (HTable) conn.getTable(table1_restore); + LOG.debug("After incremental restore: " + hTable.getTableDescriptor()); + LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows"); + Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH + ADD_ROWS); + LOG.debug("f2 has " + TEST_UTIL.countRows(hTable, fam2Name) + " rows"); + Assert.assertEquals(TEST_UTIL.countRows(hTable, fam2Name), NB_ROWS_FAM2); + hTable.close(); + + hTable = (HTable) conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 5); + hTable.close(); + + admin.close(); + conn.close(); + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java new file mode 100644 index 00000000000..59d0908340d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.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.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +/** + * 1. Create table t1, t2 + * 2. Load data to t1, t2 + * 3 Full backup t1, t2 + * 4 Delete t2 + * 5 Load data to t1 + * 6 Incremental backup t1 + */ +@Category(LargeTests.class) +public class TestIncrementalBackupDeleteTable extends TestBackupBase { + private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class); + + // implement all test cases in 1 test since incremental backup/restore has dependencies + @Test + public void TestIncBackupDeleteTable() throws Exception { + // #1 - create full backup for all tables + LOG.info("create full backup image for all tables"); + + List tables = Lists.newArrayList(table1, table2); + HBaseAdmin admin = null; + Connection conn = ConnectionFactory.createConnection(conf1); + admin = (HBaseAdmin) conn.getAdmin(); + BackupAdminImpl client = new BackupAdminImpl(conn); + + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + String backupIdFull = client.backupTables(request); + + assertTrue(checkSucceeded(backupIdFull)); + + // #2 - insert some data to table table1 + 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.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2); + t1.close(); + + // Delete table table2 + admin.disableTable(table2); + admin.deleteTable(table2); + + // #3 - incremental backup for table1 + tables = Lists.newArrayList(table1); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + String backupIdIncMultiple = client.backupTables(request); + assertTrue(checkSucceeded(backupIdIncMultiple)); + + // #4 - restore full backup for all tables, without overwrite + TableName[] tablesRestoreFull = new TableName[] { table1, table2 }; + + TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore }; + + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, + tablesRestoreFull, tablesMapFull, false)); + + // #5.1 - check tables for full restore + HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + assertTrue(hAdmin.tableExists(table2_restore)); + + // #5.2 - checking row count of tables for full restore + HTable hTable = (HTable) conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH); + hTable.close(); + + hTable = (HTable) conn.getTable(table2_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH); + hTable.close(); + + // #6 - restore incremental backup for table1 + TableName[] tablesRestoreIncMultiple = new TableName[] { table1 }; + TableName[] tablesMapIncMultiple = new TableName[] { table1_restore }; + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, + false, tablesRestoreIncMultiple, tablesMapIncMultiple, true)); + + hTable = (HTable) conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2); + hTable.close(); + admin.close(); + conn.close(); + } + +} diff --git a/hbase-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..299cd5641b0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable + * law or agreed to in writing, software distributed under the License is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License + * for the specific language governing permissions and limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@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"); + final CountDownLatch latch = new CountDownLatch(1); + final int NB_ROWS_IN_FAM3 = 6; + final byte[] fam3Name = Bytes.toBytes("f3"); + final byte[] fam2Name = Bytes.toBytes("f2"); + final Connection conn = ConnectionFactory.createConnection(conf1); + Thread t = new Thread() { + @Override + public void run() { + try { + latch.await(); + } catch (InterruptedException ie) { + } + try { + HTable t1 = (HTable) conn.getTable(table1); + Put p1; + for (int i = 0; i < NB_ROWS_IN_FAM3; i++) { + p1 = new Put(Bytes.toBytes("row-t1" + i)); + p1.addColumn(fam3Name, qualName, Bytes.toBytes("val" + i)); + t1.put(p1); + } + LOG.debug("Wrote " + NB_ROWS_IN_FAM3 + " rows into family3"); + t1.close(); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + } + }; + t.start(); + + table1Desc.addFamily(new HColumnDescriptor(fam3Name)); + // family 2 is MOB enabled + HColumnDescriptor hcd = new HColumnDescriptor(fam2Name); + hcd.setMobEnabled(true); + hcd.setMobThreshold(0L); + table1Desc.addFamily(hcd); + HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc); + + SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name); + HTable t1 = (HTable) conn.getTable(table1); + int rows0 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name); + + latch.countDown(); + String backupId = + backupTables(BackupType.FULL, Lists.newArrayList(table1), BACKUP_REMOTE_ROOT_DIR); + assertTrue(checkSucceeded(backupId)); + + LOG.info("backup complete " + backupId); + Assert.assertEquals(TEST_UTIL.countRows(t1, famName), NB_ROWS_IN_BATCH); + + t.join(); + Assert.assertEquals(TEST_UTIL.countRows(t1, fam3Name), NB_ROWS_IN_FAM3); + t1.close(); + + TableName[] tablesRestoreFull = new TableName[] { table1 }; + + TableName[] tablesMapFull = new TableName[] { table1_restore }; + + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, + tablesRestoreFull, tablesMapFull, false)); + + // check tables for full restore + HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin(); + assertTrue(hAdmin.tableExists(table1_restore)); + + // #5.2 - checking row count of tables for full restore + HTable hTable = (HTable) conn.getTable(table1_restore); + Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH); + int cnt3 = TEST_UTIL.countRows(hTable, fam3Name); + Assert.assertTrue(cnt3 >= 0 && cnt3 <= NB_ROWS_IN_FAM3); + + int rows1 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name); + Assert.assertEquals(rows0, rows1); + hTable.close(); + + hAdmin.close(); + } +} \ 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..c7ed9547e3a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable + * law or agreed to in writing, software distributed under the License is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License + * for the specific language governing permissions and limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import 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.backup.util.BackupUtils; +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 = + backupTables(BackupType.FULL, toList(table1.getNameAsString()), BACKUP_REMOTE_ROOT_DIR); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + getBackupAdmin().restore( + BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, tableset, + tablemap, false)); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(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..c61b018c7f5 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.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 = fullTableBackup(toList(table1.getNameAsString())); + LOG.info("backup complete"); + TableName[] tableset = new TableName[] { table1 }; + TableName[] tablemap = new TableName[] { table1_restore }; + getBackupAdmin().restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap, + false)); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table1_restore)); + TEST_UTIL.deleteTable(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"); + + List tables = toList(table2.getNameAsString(), table3.getNameAsString()); + String backupId = fullTableBackup(tables); + TableName[] restore_tableset = new TableName[] { table2, table3 }; + TableName[] tablemap = new TableName[] { table2_restore, table3_restore }; + getBackupAdmin().restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, restore_tableset, + tablemap, false)); + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + assertTrue(hba.tableExists(table2_restore)); + assertTrue(hba.tableExists(table3_restore)); + TEST_UTIL.deleteTable(table2_restore); + TEST_UTIL.deleteTable(table3_restore); + hba.close(); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java new file mode 100644 index 00000000000..6212ff27b16 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestSystemTableSnapshot extends TestBackupBase { + + private static final Log LOG = LogFactory.getLog(TestSystemTableSnapshot.class); + + /** + * Verify that a single table is restored to a new table + * @throws Exception + */ + //@Test - Disabled until we get resolution on system table snapshots + + public void _testBackupRestoreSystemTable() throws Exception { + + LOG.info("test snapshot system table"); + + TableName backupSystem = BackupSystemTable.getTableName(conf1); + + HBaseAdmin hba = TEST_UTIL.getHBaseAdmin(); + String snapshotName = "sysTable"; + hba.snapshot(snapshotName, backupSystem); + + hba.disableTable(backupSystem); + hba.restoreSnapshot(snapshotName); + hba.enableTable(backupSystem); + hba.close(); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java new file mode 100644 index 00000000000..229597b57a4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java @@ -0,0 +1,162 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.backup.master; + +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.TableName; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.TestBackupBase; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.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.AbstractFSWALProvider; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +@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"); + + List tableSetFullList = Lists.newArrayList(table1, table2, table3, table4); + + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + // Verify that we have no backup sessions yet + assertFalse(systemTable.hasBackupSessions()); + + List walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + List swalFiles = convert(walFiles); + BackupLogCleaner cleaner = new BackupLogCleaner(); + cleaner.setConf(TEST_UTIL.getConfiguration()); + cleaner.init(null); + cleaner.setConf(TEST_UTIL.getConfiguration()); + + Iterable deletable = cleaner.getDeletableFiles(walFiles); + int size = Iterables.size(deletable); + + // We can delete all files because we do not have yet recorded backup sessions + assertTrue(size == walFiles.size()); + + systemTable.addWALFiles(swalFiles, "backup", "root"); + String backupIdFull = fullTableBackup(tableSetFullList); + assertTrue(checkSucceeded(backupIdFull)); + // Check one more time + deletable = cleaner.getDeletableFiles(walFiles); + // We can delete wal files because they were saved into backup system table table + size = Iterables.size(deletable); + assertTrue(size == walFiles.size()); + + List newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration()); + LOG.debug("WAL list after full backup"); + 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()); + 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 + + List tableSetIncList = Lists.newArrayList(table1, table2, table3); + String backupIdIncMultiple = backupTables(BackupType.INCREMENTAL, tableSetIncList, + BACKUP_ROOT_DIR); + assertTrue(checkSucceeded(backupIdIncMultiple)); + deletable = cleaner.getDeletableFiles(newWalFiles); + + assertTrue(Iterables.size(deletable) == newWalFiles.size()); + + conn.close(); + } + } + + 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() && !AbstractFSWALProvider.isMetaFile(lfs.getPath())) { + logFiles.add(lfs); + LOG.info(lfs); + } + } + return logFiles; + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java index e71318b65a8..296b38f36cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java @@ -64,7 +64,7 @@ public class SimpleMasterProcedureManager extends MasterProcedureManager { // setup the default procedure coordinator String name = master.getServerName().toString(); ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1); - ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs( + ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator( master.getZooKeeper(), getProcedureSignature(), name); this.coordinator = new ProcedureCoordinator(comms, tpool); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java index 9a77ce5a7d6..f20c8a9c089 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java @@ -127,7 +127,7 @@ public class TestZKProcedure { String opDescription = "coordination test - " + members.length + " cohort members"; // start running the controller - ZKProcedureCoordinatorRpcs coordinatorComms = new ZKProcedureCoordinatorRpcs( + ZKProcedureCoordinator coordinatorComms = new ZKProcedureCoordinator( coordZkw, opDescription, COORDINATOR_NODE_NAME); ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE); ProcedureCoordinator coordinator = new ProcedureCoordinator(coordinatorComms, pool) { @@ -208,7 +208,7 @@ public class TestZKProcedure { // start running the coordinator and its controller ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher(); - ZKProcedureCoordinatorRpcs coordinatorController = new ZKProcedureCoordinatorRpcs( + ZKProcedureCoordinator coordinatorController = new ZKProcedureCoordinator( coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME); ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE); ProcedureCoordinator coordinator = spy(new ProcedureCoordinator(coordinatorController, pool)); @@ -393,7 +393,7 @@ public class TestZKProcedure { private void closeAll( ProcedureCoordinator coordinator, - ZKProcedureCoordinatorRpcs coordinatorController, + ZKProcedureCoordinator coordinatorController, List> cohort) throws IOException { // make sure we close all the resources diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java index d864db2993d..e7e2b23564a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.procedure; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -31,11 +31,11 @@ import java.util.concurrent.CountDownLatch; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -188,9 +188,9 @@ public class TestZKProcedureControllers { ProcedureMember member = Mockito.mock(ProcedureMember.class); - Pair> pair = controllers + Pair> pair = controllers .start(watcher, operationName, coordinator, CONTROLLER_NODE_NAME, member, expected); - ZKProcedureCoordinatorRpcs controller = pair.getFirst(); + ZKProcedureCoordinator controller = pair.getFirst(); List cohortControllers = pair.getSecond(); // start the operation Procedure p = Mockito.mock(Procedure.class); @@ -266,9 +266,9 @@ public class TestZKProcedureControllers { Procedure p = Mockito.mock(Procedure.class); Mockito.when(p.getName()).thenReturn(operationName); - Pair> pair = controllers + Pair> pair = controllers .start(watcher, operationName, coordinator, CONTROLLER_NODE_NAME, member, expected); - ZKProcedureCoordinatorRpcs controller = pair.getFirst(); + ZKProcedureCoordinator controller = pair.getFirst(); List cohortControllers = pair.getSecond(); // post 1/2 the prepare nodes early @@ -383,7 +383,7 @@ public class TestZKProcedureControllers { * Specify how the controllers that should be started (not spy/mockable) for the test. */ private abstract class StartControllers { - public abstract Pair> start( + public abstract Pair> start( ZooKeeperWatcher watcher, String operationName, ProcedureCoordinator coordinator, String controllerName, ProcedureMember member, List cohortNames) throws Exception; @@ -392,12 +392,12 @@ public class TestZKProcedureControllers { private final StartControllers startCoordinatorFirst = new StartControllers() { @Override - public Pair> start( + public Pair> start( ZooKeeperWatcher watcher, String operationName, ProcedureCoordinator coordinator, String controllerName, ProcedureMember member, List expected) throws Exception { // start the controller - ZKProcedureCoordinatorRpcs controller = new ZKProcedureCoordinatorRpcs( + ZKProcedureCoordinator controller = new ZKProcedureCoordinator( watcher, operationName, CONTROLLER_NODE_NAME); controller.start(coordinator); @@ -420,7 +420,7 @@ public class TestZKProcedureControllers { private final StartControllers startCohortFirst = new StartControllers() { @Override - public Pair> start( + public Pair> start( ZooKeeperWatcher watcher, String operationName, ProcedureCoordinator coordinator, String controllerName, ProcedureMember member, List expected) throws Exception { @@ -434,7 +434,7 @@ public class TestZKProcedureControllers { } // start the controller - ZKProcedureCoordinatorRpcs controller = new ZKProcedureCoordinatorRpcs( + ZKProcedureCoordinator controller = new ZKProcedureCoordinator( watcher, operationName, CONTROLLER_NODE_NAME); controller.start(coordinator);