From 951ec7a0b7ade9ea8aa0b112537a267b5ce693c3 Mon Sep 17 00:00:00 2001 From: Matteo Bertozzi Date: Thu, 9 Jul 2015 07:44:36 -0700 Subject: [PATCH] HBASE-13415 Procedure v2 - Use nonces for double submits from client (Stephen Yuan Jiang) Signed-off-by: Sean Busbey --- .../hadoop/hbase/client/HBaseAdmin.java | 36 +- .../hbase/protobuf/RequestConverter.java | 69 +- .../apache/hadoop/hbase/util/NonceKey.java | 65 + .../hadoop/hbase/procedure2/Procedure.java | 27 + .../hbase/procedure2/ProcedureExecutor.java | 89 +- .../hbase/procedure2/ProcedureResult.java | 34 +- .../procedure2/ProcedureTestingUtility.java | 11 +- .../procedure2/TestProcedureRecovery.java | 39 +- .../protobuf/generated/MasterProtos.java | 2102 +++++++++++++++-- .../protobuf/generated/ProcedureProtos.java | 257 +- hbase-protocol/src/main/protobuf/Master.proto | 18 + .../src/main/protobuf/Procedure.proto | 4 + .../apache/hadoop/hbase/master/HMaster.java | 107 +- .../hbase/master/MasterRpcServices.java | 52 +- .../hadoop/hbase/master/MasterServices.java | 70 +- .../regionserver/ServerNonceManager.java | 33 +- .../security/access/AccessControlLists.java | 4 +- .../visibility/VisibilityController.java | 2 +- .../hbase/client/TestHBaseAdminNoCluster.java | 5 + .../TestAssignmentManagerOnCluster.java | 2 +- .../hbase/master/TestCatalogJanitor.java | 56 +- .../MasterProcedureTestingUtility.java | 9 + .../TestAddColumnFamilyProcedure.java | 94 +- .../procedure/TestCreateTableProcedure.java | 46 +- .../TestDeleteColumnFamilyProcedure.java | 102 +- .../procedure/TestDeleteTableProcedure.java | 48 +- .../procedure/TestDisableTableProcedure.java | 44 +- .../procedure/TestEnableTableProcedure.java | 47 +- .../TestModifyColumnFamilyProcedure.java | 45 +- .../procedure/TestModifyTableProcedure.java | 27 +- .../procedure/TestTruncateTableProcedure.java | 15 +- 31 files changed, 3042 insertions(+), 517 deletions(-) create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/util/NonceKey.java 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 a06fb2cef1d..87b8278c102 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 @@ -205,6 +205,8 @@ public class HBaseAdmin implements Admin { private RpcRetryingCallerFactory rpcCallerFactory; + private NonceGenerator ng; + /** * Constructor. * See {@link #HBaseAdmin(Connection connection)} @@ -259,6 +261,8 @@ public class HBaseAdmin implements Admin { "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf); + + this.ng = this.connection.getNonceGenerator(); } @Override @@ -618,7 +622,8 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public CreateTableResponse call(int callTimeout) throws ServiceException { - CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys); + CreateTableRequest request = RequestConverter.buildCreateTableRequest( + desc, splitKeys, ng.getNonceGroup(), ng.newNonce()); return master.createTable(null, request); } }); @@ -706,7 +711,8 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public DeleteTableResponse call(int callTimeout) throws ServiceException { - DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName); + DeleteTableRequest req = + RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); return master.deleteTable(null,req); } }); @@ -829,9 +835,9 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public TruncateTableResponse call(int callTimeout) throws ServiceException { - LOG.info("Started enable of " + tableName); - TruncateTableRequest req = - RequestConverter.buildTruncateTableRequest(tableName, preserveSplits); + LOG.info("Started truncating " + tableName); + TruncateTableRequest req = RequestConverter.buildTruncateTableRequest( + tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce()); return master.truncateTable(null, req); } }); @@ -992,7 +998,8 @@ public class HBaseAdmin implements Admin { @Override public EnableTableResponse call(int callTimeout) throws ServiceException { LOG.info("Started enable of " + tableName); - EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName); + EnableTableRequest req = + RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); return master.enableTable(null,req); } }); @@ -1129,7 +1136,8 @@ public class HBaseAdmin implements Admin { @Override public DisableTableResponse call(int callTimeout) throws ServiceException { LOG.info("Started disable of " + tableName); - DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName); + DisableTableRequest req = + RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); return master.disableTable(null, req); } }); @@ -1411,7 +1419,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, columnFamily); + AddColumnRequest req = RequestConverter.buildAddColumnRequest( + tableName, columnFamily, ng.getNonceGroup(), ng.newNonce()); master.addColumn(null, req); return null; } @@ -1479,8 +1488,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - DeleteColumnRequest req = - RequestConverter.buildDeleteColumnRequest(tableName, columnFamily); + DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest( + tableName, columnFamily, ng.getNonceGroup(), ng.newNonce()); master.deleteColumn(null, req); return null; } @@ -1548,8 +1557,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - ModifyColumnRequest req = - RequestConverter.buildModifyColumnRequest(tableName, columnFamily); + ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest( + tableName, columnFamily, ng.getNonceGroup(), ng.newNonce()); master.modifyColumn(null,req); return null; } @@ -2398,7 +2407,8 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public ModifyTableResponse call(int callTimeout) throws ServiceException { - ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd); + ModifyTableRequest request = RequestConverter.buildModifyTableRequest( + tableName, htd, ng.getNonceGroup(), ng.newNonce()); return master.modifyTable(null, request); } }); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index 5a63b2344ab..96260fd358a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -1041,10 +1041,15 @@ public final class RequestConverter { * @return an AddColumnRequest */ public static AddColumnRequest buildAddColumnRequest( - final TableName tableName, final HColumnDescriptor column) { + final TableName tableName, + final HColumnDescriptor column, + final long nonceGroup, + final long nonce) { AddColumnRequest.Builder builder = AddColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); builder.setColumnFamilies(column.convert()); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1056,10 +1061,15 @@ public final class RequestConverter { * @return a DeleteColumnRequest */ public static DeleteColumnRequest buildDeleteColumnRequest( - final TableName tableName, final byte [] columnName) { + final TableName tableName, + final byte [] columnName, + final long nonceGroup, + final long nonce) { DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setColumnName(ByteStringer.wrap(columnName)); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1071,10 +1081,15 @@ public final class RequestConverter { * @return an ModifyColumnRequest */ public static ModifyColumnRequest buildModifyColumnRequest( - final TableName tableName, final HColumnDescriptor column) { + final TableName tableName, + final HColumnDescriptor column, + final long nonceGroup, + final long nonce) { ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setColumnFamilies(column.convert()); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1156,9 +1171,14 @@ public final class RequestConverter { * @param tableName * @return a DeleteTableRequest */ - public static DeleteTableRequest buildDeleteTableRequest(final TableName tableName) { + public static DeleteTableRequest buildDeleteTableRequest( + final TableName tableName, + final long nonceGroup, + final long nonce) { DeleteTableRequest.Builder builder = DeleteTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1169,11 +1189,16 @@ public final class RequestConverter { * @param preserveSplits True if the splits should be preserved * @return a TruncateTableRequest */ - public static TruncateTableRequest buildTruncateTableRequest(final TableName tableName, - boolean preserveSplits) { + public static TruncateTableRequest buildTruncateTableRequest( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) { TruncateTableRequest.Builder builder = TruncateTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); builder.setPreserveSplits(preserveSplits); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1183,9 +1208,14 @@ public final class RequestConverter { * @param tableName * @return an EnableTableRequest */ - public static EnableTableRequest buildEnableTableRequest(final TableName tableName) { + public static EnableTableRequest buildEnableTableRequest( + final TableName tableName, + final long nonceGroup, + final long nonce) { EnableTableRequest.Builder builder = EnableTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1195,9 +1225,14 @@ public final class RequestConverter { * @param tableName * @return a DisableTableRequest */ - public static DisableTableRequest buildDisableTableRequest(final TableName tableName) { + public static DisableTableRequest buildDisableTableRequest( + final TableName tableName, + final long nonceGroup, + final long nonce) { DisableTableRequest.Builder builder = DisableTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1209,7 +1244,10 @@ public final class RequestConverter { * @return a CreateTableRequest */ public static CreateTableRequest buildCreateTableRequest( - final HTableDescriptor hTableDesc, final byte [][] splitKeys) { + final HTableDescriptor hTableDesc, + final byte [][] splitKeys, + final long nonceGroup, + final long nonce) { CreateTableRequest.Builder builder = CreateTableRequest.newBuilder(); builder.setTableSchema(hTableDesc.convert()); if (splitKeys != null) { @@ -1217,6 +1255,8 @@ public final class RequestConverter { builder.addSplitKeys(ByteStringer.wrap(splitKey)); } } + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1229,10 +1269,15 @@ public final class RequestConverter { * @return a ModifyTableRequest */ public static ModifyTableRequest buildModifyTableRequest( - final TableName tableName, final HTableDescriptor hTableDesc) { + final TableName tableName, + final HTableDescriptor hTableDesc, + final long nonceGroup, + final long nonce) { ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setTableSchema(hTableDesc.convert()); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1347,7 +1392,9 @@ public final class RequestConverter { * @param synchronous * @return a SetBalancerRunningRequest */ - public static SetBalancerRunningRequest buildSetBalancerRunningRequest(boolean on, boolean synchronous) { + public static SetBalancerRunningRequest buildSetBalancerRunningRequest( + boolean on, + boolean synchronous) { return SetBalancerRunningRequest.newBuilder().setOn(on).setSynchronous(synchronous).build(); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/NonceKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/NonceKey.java new file mode 100644 index 00000000000..9c7c72ae1a5 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/NonceKey.java @@ -0,0 +1,65 @@ +/** + * + * 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.util; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + + /** + * This implementation is not smart and just treats nonce group and nonce as random bits. + */ + // TODO: we could use pure byte arrays, but then we wouldn't be able to use hash map. +@InterfaceAudience.Private +public class NonceKey { + private long group; + private long nonce; + + public NonceKey(long group, long nonce) { + assert nonce != HConstants.NO_NONCE; + this.group = group; + this.nonce = nonce; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || !(obj instanceof NonceKey)) { + return false; + } + NonceKey nk = ((NonceKey)obj); + return this.nonce == nk.nonce && this.group == nk.group; + } + + @Override + public int hashCode() { + return (int)((group >> 32) ^ group ^ (nonce >> 32) ^ nonce); + } + + @Override + public String toString() { + return "[" + group + ":" + nonce + "]"; + } + + public long getNonceGroup() { + return group; + } + + public long getNonce() { + return nonce; + } +} diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java index 601ebcd0f49..a343c8957c9 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeoutException; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.procedure2.util.StringUtils; @@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NonceKey; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -78,6 +80,8 @@ public abstract class Procedure implements Comparable { private RemoteProcedureException exception = null; private byte[] result = null; + private NonceKey nonceKey = null; + /** * The main code of the procedure. It must be idempotent since execute() * may be called multiple time in case of machine failure in the middle @@ -262,6 +266,10 @@ public abstract class Procedure implements Comparable { return parentProcId; } + public NonceKey getNonceKey() { + return nonceKey; + } + /** * @return true if the procedure has failed. * true may mean failed but not yet rolledback or failed and rolledback. @@ -413,6 +421,15 @@ public abstract class Procedure implements Comparable { this.parentProcId = parentProcId; } + /** + * Called by the ProcedureExecutor to set the value to the newly created procedure. + */ + @VisibleForTesting + @InterfaceAudience.Private + protected void setNonceKey(final NonceKey nonceKey) { + this.nonceKey = nonceKey; + } + /** * Internal method called by the ProcedureExecutor that starts the * user-level code execute(). @@ -661,6 +678,11 @@ public abstract class Procedure implements Comparable { builder.setStateData(stateStream.toByteString()); } + if (proc.getNonceKey() != null) { + builder.setNonceGroup(proc.getNonceKey().getNonceGroup()); + builder.setNonce(proc.getNonceKey().getNonce()); + } + return builder.build(); } @@ -712,6 +734,11 @@ public abstract class Procedure implements Comparable { proc.setResult(proto.getResult().toByteArray()); } + if (proto.getNonce() != HConstants.NO_NONCE) { + NonceKey nonceKey = new NonceKey(proto.getNonceGroup(), proto.getNonce()); + proc.setNonceKey(nonceKey); + } + // we want to call deserialize even when the stream is empty, mainly for testing. proc.deserializeStateData(proto.getStateData().newInput()); diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java index 49b658bf473..01e9a374813 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java @@ -39,6 +39,7 @@ 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.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; @@ -48,6 +49,7 @@ import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue; import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever; import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NonceKey; import org.apache.hadoop.hbase.util.Pair; import com.google.common.base.Preconditions; @@ -134,14 +136,17 @@ public class ProcedureExecutor { private static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min private final Map completed; + private final Map nonceKeysToProcIdsMap; private final ProcedureStore store; private final Configuration conf; public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store, - final Map completedMap) { + final Map completedMap, + final Map nonceKeysToProcIdsMap) { // set the timeout interval that triggers the periodic-procedure setTimeout(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL)); this.completed = completedMap; + this.nonceKeysToProcIdsMap = nonceKeysToProcIdsMap; this.store = store; this.conf = conf; } @@ -171,6 +176,11 @@ public class ProcedureExecutor { } store.delete(entry.getKey()); it.remove(); + + NonceKey nonceKey = result.getNonceKey(); + if (nonceKey != null) { + nonceKeysToProcIdsMap.remove(nonceKey); + } } } } @@ -224,6 +234,13 @@ public class ProcedureExecutor { private final ConcurrentHashMap procedures = new ConcurrentHashMap(); + /** + * Helper map to lookup whether the procedure already issued from the same client. + * This map contains every root procedure. + */ + private ConcurrentHashMap nonceKeysToProcIdsMap = + new ConcurrentHashMap(); + /** * Timeout Queue that contains Procedures in a WAITING_TIMEOUT state * or periodic procedures. @@ -312,6 +329,11 @@ public class ProcedureExecutor { proc.beforeReplay(getEnvironment()); procedures.put(proc.getProcId(), proc); + // add the nonce to the map + if (proc.getNonceKey() != null) { + nonceKeysToProcIdsMap.put(proc.getNonceKey(), proc.getProcId()); + } + if (proc.getState() == ProcedureState.RUNNABLE) { runnablesCount++; } @@ -343,6 +365,7 @@ public class ProcedureExecutor { assert !rollbackStack.containsKey(proc.getProcId()); procedures.remove(proc.getProcId()); completed.put(proc.getProcId(), newResultFromProcedure(proc)); + continue; } @@ -479,7 +502,8 @@ public class ProcedureExecutor { } // Add completed cleaner - waitingTimeout.add(new CompletedProcedureCleaner(conf, store, completed)); + waitingTimeout.add( + new CompletedProcedureCleaner(conf, store, completed, nonceKeysToProcIdsMap)); } public void stop() { @@ -510,6 +534,7 @@ public class ProcedureExecutor { completed.clear(); rollbackStack.clear(); procedures.clear(); + nonceKeysToProcIdsMap.clear(); waitingTimeout.clear(); runnables.clear(); lastProcId.set(-1); @@ -552,13 +577,53 @@ public class ProcedureExecutor { * @return the procedure id, that can be used to monitor the operation */ public long submitProcedure(final Procedure proc) { + return submitProcedure(proc, HConstants.NO_NONCE, HConstants.NO_NONCE); + } + + /** + * Add a new root-procedure to the executor. + * @param proc the new procedure to execute. + * @param nonceGroup + * @param nonce + * @return the procedure id, that can be used to monitor the operation + */ + public long submitProcedure( + final Procedure proc, + final long nonceGroup, + final long nonce) { Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING); Preconditions.checkArgument(isRunning()); Preconditions.checkArgument(lastProcId.get() >= 0); Preconditions.checkArgument(!proc.hasParent()); - // Initialize the Procedure ID - proc.setProcId(nextProcId()); + Long currentProcId; + + // The following part of the code has to be synchronized to prevent multiple request + // with the same nonce to execute at the same time. + synchronized (this) { + // Check whether the proc exists. If exist, just return the proc id. + // This is to prevent the same proc to submit multiple times (it could happen + // when client could not talk to server and resubmit the same request). + NonceKey noncekey = null; + if (nonce != HConstants.NO_NONCE) { + noncekey = new NonceKey(nonceGroup, nonce); + currentProcId = nonceKeysToProcIdsMap.get(noncekey); + if (currentProcId != null) { + // Found the proc + return currentProcId; + } + } + + // Initialize the Procedure ID + currentProcId = nextProcId(); + proc.setProcId(currentProcId); + + // This is new procedure. Set the noncekey and insert into the map. + if (noncekey != null) { + proc.setNonceKey(noncekey); + nonceKeysToProcIdsMap.put(noncekey, currentProcId); + } + } // end of synchronized (this) // Commit the transaction store.insert(proc, null); @@ -568,14 +633,14 @@ public class ProcedureExecutor { // Create the rollback stack for the procedure RootProcedureState stack = new RootProcedureState(); - rollbackStack.put(proc.getProcId(), stack); + rollbackStack.put(currentProcId, stack); // Submit the new subprocedures - assert !procedures.containsKey(proc.getProcId()); - procedures.put(proc.getProcId(), proc); - sendProcedureAddedNotification(proc.getProcId()); + assert !procedures.containsKey(currentProcId); + procedures.put(currentProcId, proc); + sendProcedureAddedNotification(currentProcId); runnables.addBack(proc); - return proc.getProcId(); + return currentProcId; } public ProcedureResult getResult(final long procId) { @@ -1162,8 +1227,10 @@ public class ProcedureExecutor { private static ProcedureResult newResultFromProcedure(final Procedure proc) { if (proc.isFailed()) { - return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getException()); + return new ProcedureResult( + proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getException()); } - return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getResult()); + return new ProcedureResult( + proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getResult()); } } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java index 98c293be1c3..ff5407f9c87 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.procedure2; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.NonceKey; /** * Once a Procedure completes the ProcedureExecutor takes all the useful @@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; @InterfaceAudience.Private @InterfaceStability.Evolving public class ProcedureResult { + private final NonceKey nonceKey; private final RemoteProcedureException exception; private final long lastUpdate; private final long startTime; @@ -37,21 +39,39 @@ public class ProcedureResult { private long clientAckTime = -1; - public ProcedureResult(final long startTime, final long lastUpdate, + public ProcedureResult( + final NonceKey nonceKey, + final long startTime, + final long lastUpdate, final RemoteProcedureException exception) { - this.lastUpdate = lastUpdate; - this.startTime = startTime; - this.exception = exception; - this.result = null; + this(nonceKey, exception, lastUpdate, startTime, null); } - public ProcedureResult(final long startTime, final long lastUpdate, final byte[] result) { + public ProcedureResult( + final NonceKey nonceKey, + final long startTime, + final long lastUpdate, + final byte[] result) { + this(nonceKey, null, lastUpdate, startTime, result); + } + + public ProcedureResult( + final NonceKey nonceKey, + final RemoteProcedureException exception, + final long lastUpdate, + final long startTime, + final byte[] result) { + this.nonceKey = nonceKey; + this.exception = exception; this.lastUpdate = lastUpdate; this.startTime = startTime; - this.exception = null; this.result = result; } + public NonceKey getNonceKey() { + return nonceKey; + } + public boolean isFailed() { return exception != null; } diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java index 66eb3ea6a4a..34774ed8c28 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java @@ -27,6 +27,7 @@ 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.util.Threads; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore; @@ -119,7 +120,7 @@ public class ProcedureTestingUtility { procStore.start(1); procExecutor.start(1, false); try { - return submitAndWait(procExecutor, proc); + return submitAndWait(procExecutor, proc, HConstants.NO_NONCE, HConstants.NO_NONCE); } finally { procStore.stop(false); procExecutor.stop(); @@ -127,7 +128,13 @@ public class ProcedureTestingUtility { } public static long submitAndWait(ProcedureExecutor procExecutor, Procedure proc) { - long procId = procExecutor.submitProcedure(proc); + return submitAndWait(procExecutor, proc, HConstants.NO_NONCE, HConstants.NO_NONCE); + } + + public static long submitAndWait(ProcedureExecutor procExecutor, Procedure proc, + final long nonceGroup, + final long nonce) { + long procId = procExecutor.submitProcedure(proc, nonceGroup, nonce); waitProcedure(procExecutor, procId); return procId; } diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java index 49bea5bcf22..24a448e363d 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; - import org.junit.After; import org.junit.Before; import org.junit.Assert; @@ -77,6 +76,9 @@ public class TestProcedureRecovery { procStore.start(PROCEDURE_EXECUTOR_SLOTS); procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true); procSleepInterval = 0; + + ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, false); + ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, false); } @After @@ -286,6 +288,41 @@ public class TestProcedureRecovery { ProcedureTestingUtility.assertIsAbortException(result); } + @Test(timeout=30000) + public void testCompletedProcWithSameNonce() throws Exception { + final long nonceGroup = 123; + final long nonce = 2222; + Procedure proc = new TestSingleStepProcedure(); + // Submit a proc and wait for its completion + long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc, nonceGroup, nonce); + + // Restart + restart(); + Procedure proc2 = new TestSingleStepProcedure(); + // Submit a procedure with the same nonce and expect the same procedure would return. + long procId2 = ProcedureTestingUtility.submitAndWait(procExecutor, proc2, nonceGroup, nonce); + assertTrue(procId == procId2); + + ProcedureResult result = procExecutor.getResult(procId2); + ProcedureTestingUtility.assertProcNotFailed(result); + } + + @Test(timeout=30000) + public void testRunningProcWithSameNonce() throws Exception { + final long nonceGroup = 456; + final long nonce = 33333; + Procedure proc = new TestMultiStepProcedure(); + long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc, nonceGroup, nonce); + + // Restart + restart(); + Procedure proc2 = new TestMultiStepProcedure(); + // Submit a procedure with the same nonce and expect the same procedure would return. + long procId2 = ProcedureTestingUtility.submitAndWait(procExecutor, proc2, nonceGroup, nonce); + // The original proc is not completed and the new submission should have the same proc Id. + assertTrue(procId == procId2); + } + public static class TestStateMachineProcedure extends StateMachineProcedure { enum State { STATE_1, STATE_2, STATE_3, DONE } diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java index a762d007c47..23ced4bfb8c 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java @@ -38,6 +38,26 @@ public final class MasterProtos { * required .ColumnFamilySchema column_families = 2; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnFamiliesOrBuilder(); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code AddColumnRequest} @@ -116,6 +136,16 @@ public final class MasterProtos { bitField0_ |= 0x00000002; break; } + case 24: { + bitField0_ |= 0x00000004; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -200,9 +230,43 @@ public final class MasterProtos { return columnFamilies_; } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); columnFamilies_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -238,6 +302,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeMessage(2, columnFamilies_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -255,6 +325,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, columnFamilies_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -288,6 +366,16 @@ public final class MasterProtos { result = result && getColumnFamilies() .equals(other.getColumnFamilies()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -309,6 +397,14 @@ public final class MasterProtos { hash = (37 * hash) + COLUMN_FAMILIES_FIELD_NUMBER; hash = (53 * hash) + getColumnFamilies().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -432,6 +528,10 @@ public final class MasterProtos { columnFamiliesBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -476,6 +576,14 @@ public final class MasterProtos { } else { result.columnFamilies_ = columnFamiliesBuilder_.build(); } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -498,6 +606,12 @@ public final class MasterProtos { if (other.hasColumnFamilies()) { mergeColumnFamilies(other.getColumnFamilies()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -775,6 +889,72 @@ public final class MasterProtos { return columnFamiliesBuilder_; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:AddColumnRequest) } @@ -1150,6 +1330,26 @@ public final class MasterProtos { * required bytes column_name = 2; */ com.google.protobuf.ByteString getColumnName(); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code DeleteColumnRequest} @@ -1220,6 +1420,16 @@ public final class MasterProtos { columnName_ = input.readBytes(); break; } + case 24: { + bitField0_ |= 0x00000004; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -1298,9 +1508,43 @@ public final class MasterProtos { return columnName_; } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); columnName_ = com.google.protobuf.ByteString.EMPTY; + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -1332,6 +1576,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeBytes(2, columnName_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -1349,6 +1599,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeBytesSize(2, columnName_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1382,6 +1640,16 @@ public final class MasterProtos { result = result && getColumnName() .equals(other.getColumnName()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1403,6 +1671,14 @@ public final class MasterProtos { hash = (37 * hash) + COLUMN_NAME_FIELD_NUMBER; hash = (53 * hash) + getColumnName().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -1521,6 +1797,10 @@ public final class MasterProtos { bitField0_ = (bitField0_ & ~0x00000001); columnName_ = com.google.protobuf.ByteString.EMPTY; bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -1561,6 +1841,14 @@ public final class MasterProtos { to_bitField0_ |= 0x00000002; } result.columnName_ = columnName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1583,6 +1871,12 @@ public final class MasterProtos { if (other.hasColumnName()) { setColumnName(other.getColumnName()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -1775,6 +2069,72 @@ public final class MasterProtos { return this; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:DeleteColumnRequest) } @@ -2154,6 +2514,26 @@ public final class MasterProtos { * required .ColumnFamilySchema column_families = 2; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnFamiliesOrBuilder(); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code ModifyColumnRequest} @@ -2232,6 +2612,16 @@ public final class MasterProtos { bitField0_ |= 0x00000002; break; } + case 24: { + bitField0_ |= 0x00000004; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -2316,9 +2706,43 @@ public final class MasterProtos { return columnFamilies_; } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); columnFamilies_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2354,6 +2778,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeMessage(2, columnFamilies_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -2371,6 +2801,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, columnFamilies_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2404,6 +2842,16 @@ public final class MasterProtos { result = result && getColumnFamilies() .equals(other.getColumnFamilies()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2425,6 +2873,14 @@ public final class MasterProtos { hash = (37 * hash) + COLUMN_FAMILIES_FIELD_NUMBER; hash = (53 * hash) + getColumnFamilies().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -2548,6 +3004,10 @@ public final class MasterProtos { columnFamiliesBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -2592,6 +3052,14 @@ public final class MasterProtos { } else { result.columnFamilies_ = columnFamiliesBuilder_.build(); } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -2614,6 +3082,12 @@ public final class MasterProtos { if (other.hasColumnFamilies()) { mergeColumnFamilies(other.getColumnFamilies()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2891,6 +3365,72 @@ public final class MasterProtos { return columnFamiliesBuilder_; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:ModifyColumnRequest) } @@ -8385,6 +8925,26 @@ public final class MasterProtos { * repeated bytes split_keys = 2; */ com.google.protobuf.ByteString getSplitKeys(int index); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code CreateTableRequest} @@ -8458,6 +9018,16 @@ public final class MasterProtos { splitKeys_.add(input.readBytes()); break; } + case 24: { + bitField0_ |= 0x00000002; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000004; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -8546,9 +9116,43 @@ public final class MasterProtos { return splitKeys_.get(index); } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance(); splitKeys_ = java.util.Collections.emptyList(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -8576,6 +9180,12 @@ public final class MasterProtos { for (int i = 0; i < splitKeys_.size(); i++) { output.writeBytes(2, splitKeys_.get(i)); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -8598,6 +9208,14 @@ public final class MasterProtos { size += dataSize; size += 1 * getSplitKeysList().size(); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -8628,6 +9246,16 @@ public final class MasterProtos { } result = result && getSplitKeysList() .equals(other.getSplitKeysList()); + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -8649,6 +9277,14 @@ public final class MasterProtos { hash = (37 * hash) + SPLIT_KEYS_FIELD_NUMBER; hash = (53 * hash) + getSplitKeysList().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -8767,6 +9403,10 @@ public final class MasterProtos { bitField0_ = (bitField0_ & ~0x00000001); splitKeys_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -8808,6 +9448,14 @@ public final class MasterProtos { bitField0_ = (bitField0_ & ~0x00000002); } result.splitKeys_ = splitKeys_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000002; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000004; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -8837,6 +9485,12 @@ public final class MasterProtos { } onChanged(); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -9061,6 +9715,72 @@ public final class MasterProtos { return this; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:CreateTableRequest) } @@ -9522,6 +10242,26 @@ public final class MasterProtos { * required .TableName table_name = 1; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + // optional uint64 nonce_group = 2 [default = 0]; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 3 [default = 0]; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 3 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code DeleteTableRequest} @@ -9587,6 +10327,16 @@ public final class MasterProtos { bitField0_ |= 0x00000001; break; } + case 16: { + bitField0_ |= 0x00000002; + nonceGroup_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -9649,8 +10399,42 @@ public final class MasterProtos { return tableName_; } + // optional uint64 nonce_group = 2 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 2; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 3 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 3; + private long nonce_; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -9675,6 +10459,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeMessage(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonce_); + } getUnknownFields().writeTo(output); } @@ -9688,6 +10478,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -9716,6 +10514,16 @@ public final class MasterProtos { result = result && getTableName() .equals(other.getTableName()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -9733,6 +10541,14 @@ public final class MasterProtos { hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; hash = (53 * hash) + getTableName().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -9849,6 +10665,10 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -9885,6 +10705,14 @@ public final class MasterProtos { } else { result.tableName_ = tableNameBuilder_.build(); } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -9904,6 +10732,12 @@ public final class MasterProtos { if (other.hasTableName()) { mergeTableName(other.getTableName()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -10056,6 +10890,72 @@ public final class MasterProtos { return tableNameBuilder_; } + // optional uint64 nonce_group = 2 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000002; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 3 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000004; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:DeleteTableRequest) } @@ -10527,6 +11427,26 @@ public final class MasterProtos { * optional bool preserveSplits = 2 [default = false]; */ boolean getPreserveSplits(); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code TruncateTableRequest} @@ -10597,6 +11517,16 @@ public final class MasterProtos { preserveSplits_ = input.readBool(); break; } + case 24: { + bitField0_ |= 0x00000004; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -10675,9 +11605,43 @@ public final class MasterProtos { return preserveSplits_; } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); preserveSplits_ = false; + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -10705,6 +11669,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeBool(2, preserveSplits_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -10722,6 +11692,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(2, preserveSplits_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -10755,6 +11733,16 @@ public final class MasterProtos { result = result && (getPreserveSplits() == other.getPreserveSplits()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -10776,6 +11764,14 @@ public final class MasterProtos { hash = (37 * hash) + PRESERVESPLITS_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getPreserveSplits()); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -10894,6 +11890,10 @@ public final class MasterProtos { bitField0_ = (bitField0_ & ~0x00000001); preserveSplits_ = false; bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -10934,6 +11934,14 @@ public final class MasterProtos { to_bitField0_ |= 0x00000002; } result.preserveSplits_ = preserveSplits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -10956,6 +11964,12 @@ public final class MasterProtos { if (other.hasPreserveSplits()) { setPreserveSplits(other.getPreserveSplits()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -11141,6 +12155,72 @@ public final class MasterProtos { return this; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:TruncateTableRequest) } @@ -11602,6 +12682,26 @@ public final class MasterProtos { * required .TableName table_name = 1; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + // optional uint64 nonce_group = 2 [default = 0]; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 3 [default = 0]; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 3 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code EnableTableRequest} @@ -11667,6 +12767,16 @@ public final class MasterProtos { bitField0_ |= 0x00000001; break; } + case 16: { + bitField0_ |= 0x00000002; + nonceGroup_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -11729,8 +12839,42 @@ public final class MasterProtos { return tableName_; } + // optional uint64 nonce_group = 2 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 2; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 3 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 3; + private long nonce_; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -11755,6 +12899,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeMessage(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonce_); + } getUnknownFields().writeTo(output); } @@ -11768,6 +12918,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -11796,6 +12954,16 @@ public final class MasterProtos { result = result && getTableName() .equals(other.getTableName()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -11813,6 +12981,14 @@ public final class MasterProtos { hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; hash = (53 * hash) + getTableName().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -11929,6 +13105,10 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -11965,6 +13145,14 @@ public final class MasterProtos { } else { result.tableName_ = tableNameBuilder_.build(); } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -11984,6 +13172,12 @@ public final class MasterProtos { if (other.hasTableName()) { mergeTableName(other.getTableName()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -12136,6 +13330,72 @@ public final class MasterProtos { return tableNameBuilder_; } + // optional uint64 nonce_group = 2 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000002; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 3 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000004; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:EnableTableRequest) } @@ -12597,6 +13857,26 @@ public final class MasterProtos { * required .TableName table_name = 1; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + // optional uint64 nonce_group = 2 [default = 0]; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 3 [default = 0]; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 3 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code DisableTableRequest} @@ -12662,6 +13942,16 @@ public final class MasterProtos { bitField0_ |= 0x00000001; break; } + case 16: { + bitField0_ |= 0x00000002; + nonceGroup_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -12724,8 +14014,42 @@ public final class MasterProtos { return tableName_; } + // optional uint64 nonce_group = 2 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 2; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 3 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 3; + private long nonce_; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -12750,6 +14074,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeMessage(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonce_); + } getUnknownFields().writeTo(output); } @@ -12763,6 +14093,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, tableName_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, nonceGroup_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -12791,6 +14129,16 @@ public final class MasterProtos { result = result && getTableName() .equals(other.getTableName()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -12808,6 +14156,14 @@ public final class MasterProtos { hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; hash = (53 * hash) + getTableName().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -12924,6 +14280,10 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -12960,6 +14320,14 @@ public final class MasterProtos { } else { result.tableName_ = tableNameBuilder_.build(); } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -12979,6 +14347,12 @@ public final class MasterProtos { if (other.hasTableName()) { mergeTableName(other.getTableName()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -13131,6 +14505,72 @@ public final class MasterProtos { return tableNameBuilder_; } + // optional uint64 nonce_group = 2 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000002; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 2 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 3 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000004; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 3 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:DisableTableRequest) } @@ -13606,6 +15046,26 @@ public final class MasterProtos { * required .TableSchema table_schema = 2; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder(); + + // optional uint64 nonce_group = 3 [default = 0]; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + long getNonceGroup(); + + // optional uint64 nonce = 4 [default = 0]; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 4 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code ModifyTableRequest} @@ -13684,6 +15144,16 @@ public final class MasterProtos { bitField0_ |= 0x00000002; break; } + case 24: { + bitField0_ |= 0x00000004; + nonceGroup_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -13768,9 +15238,43 @@ public final class MasterProtos { return tableSchema_; } + // optional uint64 nonce_group = 3 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 3; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 4 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 4; + private long nonce_; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance(); + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -13806,6 +15310,12 @@ public final class MasterProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeMessage(2, tableSchema_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, nonce_); + } getUnknownFields().writeTo(output); } @@ -13823,6 +15333,14 @@ public final class MasterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, tableSchema_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, nonceGroup_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -13856,6 +15374,16 @@ public final class MasterProtos { result = result && getTableSchema() .equals(other.getTableSchema()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -13877,6 +15405,14 @@ public final class MasterProtos { hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER; hash = (53 * hash) + getTableSchema().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -14000,6 +15536,10 @@ public final class MasterProtos { tableSchemaBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -14044,6 +15584,14 @@ public final class MasterProtos { } else { result.tableSchema_ = tableSchemaBuilder_.build(); } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -14066,6 +15614,12 @@ public final class MasterProtos { if (other.hasTableSchema()) { mergeTableSchema(other.getTableSchema()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -14343,6 +15897,72 @@ public final class MasterProtos { return tableSchemaBuilder_; } + // optional uint64 nonce_group = 3 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00000004; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 3 [default = 0]; + */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00000004); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 4 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00000008; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 4 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00000008); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:ModifyTableRequest) } @@ -53312,233 +54932,243 @@ public final class MasterProtos { java.lang.String[] descriptorData = { "\n\014Master.proto\032\013HBase.proto\032\014Client.prot" + "o\032\023ClusterStatus.proto\032\023ErrorHandling.pr" + - "oto\032\013Quota.proto\"`\n\020AddColumnRequest\022\036\n\n" + - "table_name\030\001 \002(\0132\n.TableName\022,\n\017column_f" + - "amilies\030\002 \002(\0132\023.ColumnFamilySchema\"\023\n\021Ad" + - "dColumnResponse\"J\n\023DeleteColumnRequest\022\036" + - "\n\ntable_name\030\001 \002(\0132\n.TableName\022\023\n\013column" + - "_name\030\002 \002(\014\"\026\n\024DeleteColumnResponse\"c\n\023M" + - "odifyColumnRequest\022\036\n\ntable_name\030\001 \002(\0132\n" + - ".TableName\022,\n\017column_families\030\002 \002(\0132\023.Co", - "lumnFamilySchema\"\026\n\024ModifyColumnResponse" + - "\"\\\n\021MoveRegionRequest\022 \n\006region\030\001 \002(\0132\020." + - "RegionSpecifier\022%\n\020dest_server_name\030\002 \001(" + - "\0132\013.ServerName\"\024\n\022MoveRegionResponse\"\200\001\n" + - "\035DispatchMergingRegionsRequest\022\"\n\010region" + - "_a\030\001 \002(\0132\020.RegionSpecifier\022\"\n\010region_b\030\002" + - " \002(\0132\020.RegionSpecifier\022\027\n\010forcible\030\003 \001(\010" + - ":\005false\" \n\036DispatchMergingRegionsRespons" + - "e\"7\n\023AssignRegionRequest\022 \n\006region\030\001 \002(\013" + - "2\020.RegionSpecifier\"\026\n\024AssignRegionRespon", - "se\"O\n\025UnassignRegionRequest\022 \n\006region\030\001 " + - "\002(\0132\020.RegionSpecifier\022\024\n\005force\030\002 \001(\010:\005fa" + - "lse\"\030\n\026UnassignRegionResponse\"8\n\024Offline" + - "RegionRequest\022 \n\006region\030\001 \002(\0132\020.RegionSp" + - "ecifier\"\027\n\025OfflineRegionResponse\"L\n\022Crea" + - "teTableRequest\022\"\n\014table_schema\030\001 \002(\0132\014.T" + - "ableSchema\022\022\n\nsplit_keys\030\002 \003(\014\"&\n\023Create" + - "TableResponse\022\017\n\007proc_id\030\001 \001(\004\"4\n\022Delete" + - "TableRequest\022\036\n\ntable_name\030\001 \002(\0132\n.Table" + - "Name\"&\n\023DeleteTableResponse\022\017\n\007proc_id\030\001", - " \001(\004\"T\n\024TruncateTableRequest\022\035\n\ttableNam" + - "e\030\001 \002(\0132\n.TableName\022\035\n\016preserveSplits\030\002 " + - "\001(\010:\005false\"(\n\025TruncateTableResponse\022\017\n\007p" + - "roc_id\030\001 \001(\004\"4\n\022EnableTableRequest\022\036\n\nta" + - "ble_name\030\001 \002(\0132\n.TableName\"&\n\023EnableTabl" + - "eResponse\022\017\n\007proc_id\030\001 \001(\004\"5\n\023DisableTab" + - "leRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableNam" + - "e\"\'\n\024DisableTableResponse\022\017\n\007proc_id\030\001 \001" + - "(\004\"X\n\022ModifyTableRequest\022\036\n\ntable_name\030\001" + - " \002(\0132\n.TableName\022\"\n\014table_schema\030\002 \002(\0132\014", - ".TableSchema\"&\n\023ModifyTableResponse\022\017\n\007p" + - "roc_id\030\001 \001(\004\"K\n\026CreateNamespaceRequest\0221" + - "\n\023namespaceDescriptor\030\001 \002(\0132\024.NamespaceD" + - "escriptor\"\031\n\027CreateNamespaceResponse\"/\n\026" + - "DeleteNamespaceRequest\022\025\n\rnamespaceName\030" + - "\001 \002(\t\"\031\n\027DeleteNamespaceResponse\"K\n\026Modi" + - "fyNamespaceRequest\0221\n\023namespaceDescripto" + - "r\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027ModifyN" + - "amespaceResponse\"6\n\035GetNamespaceDescript" + - "orRequest\022\025\n\rnamespaceName\030\001 \002(\t\"S\n\036GetN", - "amespaceDescriptorResponse\0221\n\023namespaceD" + - "escriptor\030\001 \002(\0132\024.NamespaceDescriptor\"!\n" + - "\037ListNamespaceDescriptorsRequest\"U\n List" + - "NamespaceDescriptorsResponse\0221\n\023namespac" + - "eDescriptor\030\001 \003(\0132\024.NamespaceDescriptor\"" + - "?\n&ListTableDescriptorsByNamespaceReques" + - "t\022\025\n\rnamespaceName\030\001 \002(\t\"L\n\'ListTableDes" + - "criptorsByNamespaceResponse\022!\n\013tableSche" + - "ma\030\001 \003(\0132\014.TableSchema\"9\n ListTableNames" + - "ByNamespaceRequest\022\025\n\rnamespaceName\030\001 \002(", - "\t\"B\n!ListTableNamesByNamespaceResponse\022\035" + - "\n\ttableName\030\001 \003(\0132\n.TableName\"\021\n\017Shutdow" + - "nRequest\"\022\n\020ShutdownResponse\"\023\n\021StopMast" + - "erRequest\"\024\n\022StopMasterResponse\"\020\n\016Balan" + - "ceRequest\"\'\n\017BalanceResponse\022\024\n\014balancer" + - "_ran\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022" + - "\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetB" + - "alancerRunningResponse\022\032\n\022prev_balance_v" + - "alue\030\001 \001(\010\"\032\n\030IsBalancerEnabledRequest\"," + - "\n\031IsBalancerEnabledResponse\022\017\n\007enabled\030\001", - " \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026RunCata" + - "logScanResponse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033" + - "EnableCatalogJanitorRequest\022\016\n\006enable\030\001 " + - "\002(\010\"2\n\034EnableCatalogJanitorResponse\022\022\n\np" + - "rev_value\030\001 \001(\010\" \n\036IsCatalogJanitorEnabl" + - "edRequest\"0\n\037IsCatalogJanitorEnabledResp" + - "onse\022\r\n\005value\030\001 \002(\010\"9\n\017SnapshotRequest\022&" + - "\n\010snapshot\030\001 \002(\0132\024.SnapshotDescription\"," + - "\n\020SnapshotResponse\022\030\n\020expected_timeout\030\001" + - " \002(\003\"\036\n\034GetCompletedSnapshotsRequest\"H\n\035", - "GetCompletedSnapshotsResponse\022\'\n\tsnapsho" + - "ts\030\001 \003(\0132\024.SnapshotDescription\"?\n\025Delete" + - "SnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snap" + - "shotDescription\"\030\n\026DeleteSnapshotRespons" + - "e\"@\n\026RestoreSnapshotRequest\022&\n\010snapshot\030" + - "\001 \002(\0132\024.SnapshotDescription\"\031\n\027RestoreSn" + - "apshotResponse\"?\n\025IsSnapshotDoneRequest\022" + - "&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescription\"" + - "U\n\026IsSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:" + - "\005false\022&\n\010snapshot\030\002 \001(\0132\024.SnapshotDescr", - "iption\"F\n\034IsRestoreSnapshotDoneRequest\022&" + - "\n\010snapshot\030\001 \001(\0132\024.SnapshotDescription\"4" + - "\n\035IsRestoreSnapshotDoneResponse\022\023\n\004done\030" + - "\001 \001(\010:\005false\"=\n\033GetSchemaAlterStatusRequ" + - "est\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"T\n\034G" + - "etSchemaAlterStatusResponse\022\035\n\025yet_to_up" + - "date_regions\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(" + - "\r\"\202\001\n\032GetTableDescriptorsRequest\022\037\n\013tabl" + - "e_names\030\001 \003(\0132\n.TableName\022\r\n\005regex\030\002 \001(\t" + - "\022!\n\022include_sys_tables\030\003 \001(\010:\005false\022\021\n\tn", - "amespace\030\004 \001(\t\"A\n\033GetTableDescriptorsRes" + - "ponse\022\"\n\014table_schema\030\001 \003(\0132\014.TableSchem" + - "a\"[\n\024GetTableNamesRequest\022\r\n\005regex\030\001 \001(\t" + - "\022!\n\022include_sys_tables\030\002 \001(\010:\005false\022\021\n\tn" + - "amespace\030\003 \001(\t\"8\n\025GetTableNamesResponse\022" + - "\037\n\013table_names\030\001 \003(\0132\n.TableName\"6\n\024GetT" + - "ableStateRequest\022\036\n\ntable_name\030\001 \002(\0132\n.T" + - "ableName\"9\n\025GetTableStateResponse\022 \n\013tab" + - "le_state\030\001 \002(\0132\013.TableState\"\031\n\027GetCluste" + - "rStatusRequest\"B\n\030GetClusterStatusRespon", - "se\022&\n\016cluster_status\030\001 \002(\0132\016.ClusterStat" + - "us\"\030\n\026IsMasterRunningRequest\"4\n\027IsMaster" + - "RunningResponse\022\031\n\021is_master_running\030\001 \002" + - "(\010\"@\n\024ExecProcedureRequest\022(\n\tprocedure\030" + - "\001 \002(\0132\025.ProcedureDescription\"F\n\025ExecProc" + - "edureResponse\022\030\n\020expected_timeout\030\001 \001(\003\022" + - "\023\n\013return_data\030\002 \001(\014\"B\n\026IsProcedureDoneR" + - "equest\022(\n\tprocedure\030\001 \001(\0132\025.ProcedureDes" + - "cription\"W\n\027IsProcedureDoneResponse\022\023\n\004d" + - "one\030\001 \001(\010:\005false\022\'\n\010snapshot\030\002 \001(\0132\025.Pro", - "cedureDescription\",\n\031GetProcedureResultR" + - "equest\022\017\n\007proc_id\030\001 \002(\004\"\347\001\n\032GetProcedure" + - "ResultResponse\0220\n\005state\030\001 \002(\0162!.GetProce" + - "dureResultResponse.State\022\022\n\nstart_time\030\002" + - " \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(" + - "\014\022+\n\texception\030\005 \001(\0132\030.ForeignExceptionM" + - "essage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNIN" + - "G\020\001\022\014\n\010FINISHED\020\002\"\273\001\n\017SetQuotaRequest\022\021\n" + - "\tuser_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\t" + - "namespace\030\003 \001(\t\022\036\n\ntable_name\030\004 \001(\0132\n.Ta", - "bleName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_gl" + - "obals\030\006 \001(\010\022\"\n\010throttle\030\007 \001(\0132\020.Throttle" + - "Request\"\022\n\020SetQuotaResponse\"A\n\037MajorComp" + - "actionTimestampRequest\022\036\n\ntable_name\030\001 \002" + - "(\0132\n.TableName\"L\n(MajorCompactionTimesta" + - "mpForRegionRequest\022 \n\006region\030\001 \002(\0132\020.Reg" + - "ionSpecifier\"@\n MajorCompactionTimestamp" + - "Response\022\034\n\024compaction_timestamp\030\001 \002(\0032\343" + - "\033\n\rMasterService\022S\n\024GetSchemaAlterStatus" + - "\022\034.GetSchemaAlterStatusRequest\032\035.GetSche", - "maAlterStatusResponse\022P\n\023GetTableDescrip" + - "tors\022\033.GetTableDescriptorsRequest\032\034.GetT" + - "ableDescriptorsResponse\022>\n\rGetTableNames" + - "\022\025.GetTableNamesRequest\032\026.GetTableNamesR" + - "esponse\022G\n\020GetClusterStatus\022\030.GetCluster" + - "StatusRequest\032\031.GetClusterStatusResponse" + - "\022D\n\017IsMasterRunning\022\027.IsMasterRunningReq" + - "uest\032\030.IsMasterRunningResponse\0222\n\tAddCol" + - "umn\022\021.AddColumnRequest\032\022.AddColumnRespon" + - "se\022;\n\014DeleteColumn\022\024.DeleteColumnRequest", - "\032\025.DeleteColumnResponse\022;\n\014ModifyColumn\022" + - "\024.ModifyColumnRequest\032\025.ModifyColumnResp" + - "onse\0225\n\nMoveRegion\022\022.MoveRegionRequest\032\023" + - ".MoveRegionResponse\022Y\n\026DispatchMergingRe" + - "gions\022\036.DispatchMergingRegionsRequest\032\037." + - "DispatchMergingRegionsResponse\022;\n\014Assign" + - "Region\022\024.AssignRegionRequest\032\025.AssignReg" + - "ionResponse\022A\n\016UnassignRegion\022\026.Unassign" + - "RegionRequest\032\027.UnassignRegionResponse\022>" + - "\n\rOfflineRegion\022\025.OfflineRegionRequest\032\026", - ".OfflineRegionResponse\0228\n\013DeleteTable\022\023." + - "DeleteTableRequest\032\024.DeleteTableResponse" + - "\022>\n\rtruncateTable\022\025.TruncateTableRequest" + - "\032\026.TruncateTableResponse\0228\n\013EnableTable\022" + - "\023.EnableTableRequest\032\024.EnableTableRespon" + - "se\022;\n\014DisableTable\022\024.DisableTableRequest" + - "\032\025.DisableTableResponse\0228\n\013ModifyTable\022\023" + - ".ModifyTableRequest\032\024.ModifyTableRespons" + - "e\0228\n\013CreateTable\022\023.CreateTableRequest\032\024." + - "CreateTableResponse\022/\n\010Shutdown\022\020.Shutdo", - "wnRequest\032\021.ShutdownResponse\0225\n\nStopMast" + - "er\022\022.StopMasterRequest\032\023.StopMasterRespo" + - "nse\022,\n\007Balance\022\017.BalanceRequest\032\020.Balanc" + - "eResponse\022M\n\022SetBalancerRunning\022\032.SetBal" + - "ancerRunningRequest\032\033.SetBalancerRunning" + - "Response\022J\n\021IsBalancerEnabled\022\031.IsBalanc" + - "erEnabledRequest\032\032.IsBalancerEnabledResp" + - "onse\022A\n\016RunCatalogScan\022\026.RunCatalogScanR" + - "equest\032\027.RunCatalogScanResponse\022S\n\024Enabl" + - "eCatalogJanitor\022\034.EnableCatalogJanitorRe", - "quest\032\035.EnableCatalogJanitorResponse\022\\\n\027" + - "IsCatalogJanitorEnabled\022\037.IsCatalogJanit" + - "orEnabledRequest\032 .IsCatalogJanitorEnabl" + - "edResponse\022L\n\021ExecMasterService\022\032.Coproc" + - "essorServiceRequest\032\033.CoprocessorService" + - "Response\022/\n\010Snapshot\022\020.SnapshotRequest\032\021" + - ".SnapshotResponse\022V\n\025GetCompletedSnapsho" + - "ts\022\035.GetCompletedSnapshotsRequest\032\036.GetC" + - "ompletedSnapshotsResponse\022A\n\016DeleteSnaps" + - "hot\022\026.DeleteSnapshotRequest\032\027.DeleteSnap", - "shotResponse\022A\n\016IsSnapshotDone\022\026.IsSnaps" + - "hotDoneRequest\032\027.IsSnapshotDoneResponse\022" + - "D\n\017RestoreSnapshot\022\027.RestoreSnapshotRequ" + - "est\032\030.RestoreSnapshotResponse\022V\n\025IsResto" + - "reSnapshotDone\022\035.IsRestoreSnapshotDoneRe" + - "quest\032\036.IsRestoreSnapshotDoneResponse\022>\n" + - "\rExecProcedure\022\025.ExecProcedureRequest\032\026." + - "ExecProcedureResponse\022E\n\024ExecProcedureWi" + - "thRet\022\025.ExecProcedureRequest\032\026.ExecProce" + - "dureResponse\022D\n\017IsProcedureDone\022\027.IsProc", - "edureDoneRequest\032\030.IsProcedureDoneRespon" + - "se\022D\n\017ModifyNamespace\022\027.ModifyNamespaceR" + - "equest\032\030.ModifyNamespaceResponse\022D\n\017Crea" + - "teNamespace\022\027.CreateNamespaceRequest\032\030.C" + - "reateNamespaceResponse\022D\n\017DeleteNamespac" + - "e\022\027.DeleteNamespaceRequest\032\030.DeleteNames" + - "paceResponse\022Y\n\026GetNamespaceDescriptor\022\036" + - ".GetNamespaceDescriptorRequest\032\037.GetName" + - "spaceDescriptorResponse\022_\n\030ListNamespace" + - "Descriptors\022 .ListNamespaceDescriptorsRe", - "quest\032!.ListNamespaceDescriptorsResponse" + - "\022t\n\037ListTableDescriptorsByNamespace\022\'.Li" + - "stTableDescriptorsByNamespaceRequest\032(.L" + - "istTableDescriptorsByNamespaceResponse\022b" + - "\n\031ListTableNamesByNamespace\022!.ListTableN" + - "amesByNamespaceRequest\032\".ListTableNamesB" + - "yNamespaceResponse\022>\n\rGetTableState\022\025.Ge" + - "tTableStateRequest\032\026.GetTableStateRespon" + - "se\022/\n\010SetQuota\022\020.SetQuotaRequest\032\021.SetQu" + - "otaResponse\022f\n\037getLastMajorCompactionTim", - "estamp\022 .MajorCompactionTimestampRequest" + - "\032!.MajorCompactionTimestampResponse\022x\n(g" + - "etLastMajorCompactionTimestampForRegion\022" + - ").MajorCompactionTimestampForRegionReque" + - "st\032!.MajorCompactionTimestampResponse\022M\n" + - "\022getProcedureResult\022\032.GetProcedureResult" + - "Request\032\033.GetProcedureResultResponseBB\n*" + - "org.apache.hadoop.hbase.protobuf.generat" + - "edB\014MasterProtosH\001\210\001\001\240\001\001" + "oto\032\013Quota.proto\"\212\001\n\020AddColumnRequest\022\036\n" + + "\ntable_name\030\001 \002(\0132\n.TableName\022,\n\017column_" + + "families\030\002 \002(\0132\023.ColumnFamilySchema\022\026\n\013n" + + "once_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\023\n" + + "\021AddColumnResponse\"t\n\023DeleteColumnReques" + + "t\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022\023\n\013col" + + "umn_name\030\002 \002(\014\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020" + + "\n\005nonce\030\004 \001(\004:\0010\"\026\n\024DeleteColumnResponse", + "\"\215\001\n\023ModifyColumnRequest\022\036\n\ntable_name\030\001" + + " \002(\0132\n.TableName\022,\n\017column_families\030\002 \002(" + + "\0132\023.ColumnFamilySchema\022\026\n\013nonce_group\030\003 " + + "\001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\026\n\024ModifyColumn" + + "Response\"\\\n\021MoveRegionRequest\022 \n\006region\030" + + "\001 \002(\0132\020.RegionSpecifier\022%\n\020dest_server_n" + + "ame\030\002 \001(\0132\013.ServerName\"\024\n\022MoveRegionResp" + + "onse\"\200\001\n\035DispatchMergingRegionsRequest\022\"" + + "\n\010region_a\030\001 \002(\0132\020.RegionSpecifier\022\"\n\010re" + + "gion_b\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcib", + "le\030\003 \001(\010:\005false\" \n\036DispatchMergingRegion" + + "sResponse\"7\n\023AssignRegionRequest\022 \n\006regi" + + "on\030\001 \002(\0132\020.RegionSpecifier\"\026\n\024AssignRegi" + + "onResponse\"O\n\025UnassignRegionRequest\022 \n\006r" + + "egion\030\001 \002(\0132\020.RegionSpecifier\022\024\n\005force\030\002" + + " \001(\010:\005false\"\030\n\026UnassignRegionResponse\"8\n" + + "\024OfflineRegionRequest\022 \n\006region\030\001 \002(\0132\020." + + "RegionSpecifier\"\027\n\025OfflineRegionResponse" + + "\"v\n\022CreateTableRequest\022\"\n\014table_schema\030\001" + + " \002(\0132\014.TableSchema\022\022\n\nsplit_keys\030\002 \003(\014\022\026", + "\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010" + + "\"&\n\023CreateTableResponse\022\017\n\007proc_id\030\001 \001(\004" + + "\"^\n\022DeleteTableRequest\022\036\n\ntable_name\030\001 \002" + + "(\0132\n.TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020" + + "\n\005nonce\030\003 \001(\004:\0010\"&\n\023DeleteTableResponse\022" + + "\017\n\007proc_id\030\001 \001(\004\"~\n\024TruncateTableRequest" + + "\022\035\n\ttableName\030\001 \002(\0132\n.TableName\022\035\n\016prese" + + "rveSplits\030\002 \001(\010:\005false\022\026\n\013nonce_group\030\003 " + + "\001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"(\n\025TruncateTabl" + + "eResponse\022\017\n\007proc_id\030\001 \001(\004\"^\n\022EnableTabl", + "eRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableName" + + "\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:" + + "\0010\"&\n\023EnableTableResponse\022\017\n\007proc_id\030\001 \001" + + "(\004\"_\n\023DisableTableRequest\022\036\n\ntable_name\030" + + "\001 \002(\0132\n.TableName\022\026\n\013nonce_group\030\002 \001(\004:\001" + + "0\022\020\n\005nonce\030\003 \001(\004:\0010\"\'\n\024DisableTableRespo" + + "nse\022\017\n\007proc_id\030\001 \001(\004\"\202\001\n\022ModifyTableRequ" + + "est\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022\"\n\014t" + + "able_schema\030\002 \002(\0132\014.TableSchema\022\026\n\013nonce" + + "_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023Mod", + "ifyTableResponse\022\017\n\007proc_id\030\001 \001(\004\"K\n\026Cre" + + "ateNamespaceRequest\0221\n\023namespaceDescript" + + "or\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027Create" + + "NamespaceResponse\"/\n\026DeleteNamespaceRequ" + + "est\022\025\n\rnamespaceName\030\001 \002(\t\"\031\n\027DeleteName" + + "spaceResponse\"K\n\026ModifyNamespaceRequest\022" + + "1\n\023namespaceDescriptor\030\001 \002(\0132\024.Namespace" + + "Descriptor\"\031\n\027ModifyNamespaceResponse\"6\n" + + "\035GetNamespaceDescriptorRequest\022\025\n\rnamesp" + + "aceName\030\001 \002(\t\"S\n\036GetNamespaceDescriptorR", + "esponse\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.N" + + "amespaceDescriptor\"!\n\037ListNamespaceDescr" + + "iptorsRequest\"U\n ListNamespaceDescriptor" + + "sResponse\0221\n\023namespaceDescriptor\030\001 \003(\0132\024" + + ".NamespaceDescriptor\"?\n&ListTableDescrip" + + "torsByNamespaceRequest\022\025\n\rnamespaceName\030" + + "\001 \002(\t\"L\n\'ListTableDescriptorsByNamespace" + + "Response\022!\n\013tableSchema\030\001 \003(\0132\014.TableSch" + + "ema\"9\n ListTableNamesByNamespaceRequest\022" + + "\025\n\rnamespaceName\030\001 \002(\t\"B\n!ListTableNames", + "ByNamespaceResponse\022\035\n\ttableName\030\001 \003(\0132\n" + + ".TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutdow" + + "nResponse\"\023\n\021StopMasterRequest\"\024\n\022StopMa" + + "sterResponse\"\020\n\016BalanceRequest\"\'\n\017Balanc" + + "eResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031SetBa" + + "lancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013sync" + + "hronous\030\002 \001(\010\"8\n\032SetBalancerRunningRespo" + + "nse\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030IsBal" + + "ancerEnabledRequest\",\n\031IsBalancerEnabled" + + "Response\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCatalogS", + "canRequest\"-\n\026RunCatalogScanResponse\022\023\n\013" + + "scan_result\030\001 \001(\005\"-\n\033EnableCatalogJanito" + + "rRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034EnableCatalo" + + "gJanitorResponse\022\022\n\nprev_value\030\001 \001(\010\" \n\036" + + "IsCatalogJanitorEnabledRequest\"0\n\037IsCata" + + "logJanitorEnabledResponse\022\r\n\005value\030\001 \002(\010" + + "\"9\n\017SnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024." + + "SnapshotDescription\",\n\020SnapshotResponse\022" + + "\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034GetComplete" + + "dSnapshotsRequest\"H\n\035GetCompletedSnapsho", + "tsResponse\022\'\n\tsnapshots\030\001 \003(\0132\024.Snapshot" + + "Description\"?\n\025DeleteSnapshotRequest\022&\n\010" + + "snapshot\030\001 \002(\0132\024.SnapshotDescription\"\030\n\026" + + "DeleteSnapshotResponse\"@\n\026RestoreSnapsho" + + "tRequest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDes" + + "cription\"\031\n\027RestoreSnapshotResponse\"?\n\025I" + + "sSnapshotDoneRequest\022&\n\010snapshot\030\001 \001(\0132\024" + + ".SnapshotDescription\"U\n\026IsSnapshotDoneRe" + + "sponse\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010snapshot\030" + + "\002 \001(\0132\024.SnapshotDescription\"F\n\034IsRestore", + "SnapshotDoneRequest\022&\n\010snapshot\030\001 \001(\0132\024." + + "SnapshotDescription\"4\n\035IsRestoreSnapshot" + + "DoneResponse\022\023\n\004done\030\001 \001(\010:\005false\"=\n\033Get" + + "SchemaAlterStatusRequest\022\036\n\ntable_name\030\001" + + " \002(\0132\n.TableName\"T\n\034GetSchemaAlterStatus" + + "Response\022\035\n\025yet_to_update_regions\030\001 \001(\r\022" + + "\025\n\rtotal_regions\030\002 \001(\r\"\202\001\n\032GetTableDescr" + + "iptorsRequest\022\037\n\013table_names\030\001 \003(\0132\n.Tab" + + "leName\022\r\n\005regex\030\002 \001(\t\022!\n\022include_sys_tab" + + "les\030\003 \001(\010:\005false\022\021\n\tnamespace\030\004 \001(\t\"A\n\033G", + "etTableDescriptorsResponse\022\"\n\014table_sche" + + "ma\030\001 \003(\0132\014.TableSchema\"[\n\024GetTableNamesR" + + "equest\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_tab" + + "les\030\002 \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"8\n\025G" + + "etTableNamesResponse\022\037\n\013table_names\030\001 \003(" + + "\0132\n.TableName\"6\n\024GetTableStateRequest\022\036\n" + + "\ntable_name\030\001 \002(\0132\n.TableName\"9\n\025GetTabl" + + "eStateResponse\022 \n\013table_state\030\001 \002(\0132\013.Ta" + + "bleState\"\031\n\027GetClusterStatusRequest\"B\n\030G" + + "etClusterStatusResponse\022&\n\016cluster_statu", + "s\030\001 \002(\0132\016.ClusterStatus\"\030\n\026IsMasterRunni" + + "ngRequest\"4\n\027IsMasterRunningResponse\022\031\n\021" + + "is_master_running\030\001 \002(\010\"@\n\024ExecProcedure" + + "Request\022(\n\tprocedure\030\001 \002(\0132\025.ProcedureDe" + + "scription\"F\n\025ExecProcedureResponse\022\030\n\020ex" + + "pected_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(" + + "\014\"B\n\026IsProcedureDoneRequest\022(\n\tprocedure" + + "\030\001 \001(\0132\025.ProcedureDescription\"W\n\027IsProce" + + "dureDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022\'\n" + + "\010snapshot\030\002 \001(\0132\025.ProcedureDescription\",", + "\n\031GetProcedureResultRequest\022\017\n\007proc_id\030\001" + + " \002(\004\"\347\001\n\032GetProcedureResultResponse\0220\n\005s" + + "tate\030\001 \002(\0162!.GetProcedureResultResponse." + + "State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_update" + + "\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\022+\n\texception\030\005 \001(" + + "\0132\030.ForeignExceptionMessage\"1\n\005State\022\r\n\t" + + "NOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"\273" + + "\001\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n" + + "\nuser_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\036\n\n" + + "table_name\030\004 \001(\0132\n.TableName\022\022\n\nremove_a", + "ll\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022\"\n\010thro" + + "ttle\030\007 \001(\0132\020.ThrottleRequest\"\022\n\020SetQuota" + + "Response\"A\n\037MajorCompactionTimestampRequ" + + "est\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"L\n(M" + + "ajorCompactionTimestampForRegionRequest\022" + + " \n\006region\030\001 \002(\0132\020.RegionSpecifier\"@\n Maj" + + "orCompactionTimestampResponse\022\034\n\024compact" + + "ion_timestamp\030\001 \002(\0032\343\033\n\rMasterService\022S\n" + + "\024GetSchemaAlterStatus\022\034.GetSchemaAlterSt" + + "atusRequest\032\035.GetSchemaAlterStatusRespon", + "se\022P\n\023GetTableDescriptors\022\033.GetTableDesc" + + "riptorsRequest\032\034.GetTableDescriptorsResp" + + "onse\022>\n\rGetTableNames\022\025.GetTableNamesReq" + + "uest\032\026.GetTableNamesResponse\022G\n\020GetClust" + + "erStatus\022\030.GetClusterStatusRequest\032\031.Get" + + "ClusterStatusResponse\022D\n\017IsMasterRunning" + + "\022\027.IsMasterRunningRequest\032\030.IsMasterRunn" + + "ingResponse\0222\n\tAddColumn\022\021.AddColumnRequ" + + "est\032\022.AddColumnResponse\022;\n\014DeleteColumn\022" + + "\024.DeleteColumnRequest\032\025.DeleteColumnResp", + "onse\022;\n\014ModifyColumn\022\024.ModifyColumnReque" + + "st\032\025.ModifyColumnResponse\0225\n\nMoveRegion\022" + + "\022.MoveRegionRequest\032\023.MoveRegionResponse" + + "\022Y\n\026DispatchMergingRegions\022\036.DispatchMer" + + "gingRegionsRequest\032\037.DispatchMergingRegi" + + "onsResponse\022;\n\014AssignRegion\022\024.AssignRegi" + + "onRequest\032\025.AssignRegionResponse\022A\n\016Unas" + + "signRegion\022\026.UnassignRegionRequest\032\027.Una" + + "ssignRegionResponse\022>\n\rOfflineRegion\022\025.O" + + "fflineRegionRequest\032\026.OfflineRegionRespo", + "nse\0228\n\013DeleteTable\022\023.DeleteTableRequest\032" + + "\024.DeleteTableResponse\022>\n\rtruncateTable\022\025" + + ".TruncateTableRequest\032\026.TruncateTableRes" + + "ponse\0228\n\013EnableTable\022\023.EnableTableReques" + + "t\032\024.EnableTableResponse\022;\n\014DisableTable\022" + + "\024.DisableTableRequest\032\025.DisableTableResp" + + "onse\0228\n\013ModifyTable\022\023.ModifyTableRequest" + + "\032\024.ModifyTableResponse\0228\n\013CreateTable\022\023." + + "CreateTableRequest\032\024.CreateTableResponse" + + "\022/\n\010Shutdown\022\020.ShutdownRequest\032\021.Shutdow", + "nResponse\0225\n\nStopMaster\022\022.StopMasterRequ" + + "est\032\023.StopMasterResponse\022,\n\007Balance\022\017.Ba" + + "lanceRequest\032\020.BalanceResponse\022M\n\022SetBal" + + "ancerRunning\022\032.SetBalancerRunningRequest" + + "\032\033.SetBalancerRunningResponse\022J\n\021IsBalan" + + "cerEnabled\022\031.IsBalancerEnabledRequest\032\032." + + "IsBalancerEnabledResponse\022A\n\016RunCatalogS" + + "can\022\026.RunCatalogScanRequest\032\027.RunCatalog" + + "ScanResponse\022S\n\024EnableCatalogJanitor\022\034.E" + + "nableCatalogJanitorRequest\032\035.EnableCatal", + "ogJanitorResponse\022\\\n\027IsCatalogJanitorEna" + + "bled\022\037.IsCatalogJanitorEnabledRequest\032 ." + + "IsCatalogJanitorEnabledResponse\022L\n\021ExecM" + + "asterService\022\032.CoprocessorServiceRequest" + + "\032\033.CoprocessorServiceResponse\022/\n\010Snapsho" + + "t\022\020.SnapshotRequest\032\021.SnapshotResponse\022V" + + "\n\025GetCompletedSnapshots\022\035.GetCompletedSn" + + "apshotsRequest\032\036.GetCompletedSnapshotsRe" + + "sponse\022A\n\016DeleteSnapshot\022\026.DeleteSnapsho" + + "tRequest\032\027.DeleteSnapshotResponse\022A\n\016IsS", + "napshotDone\022\026.IsSnapshotDoneRequest\032\027.Is" + + "SnapshotDoneResponse\022D\n\017RestoreSnapshot\022" + + "\027.RestoreSnapshotRequest\032\030.RestoreSnapsh" + + "otResponse\022V\n\025IsRestoreSnapshotDone\022\035.Is" + + "RestoreSnapshotDoneRequest\032\036.IsRestoreSn" + + "apshotDoneResponse\022>\n\rExecProcedure\022\025.Ex" + + "ecProcedureRequest\032\026.ExecProcedureRespon" + + "se\022E\n\024ExecProcedureWithRet\022\025.ExecProcedu" + + "reRequest\032\026.ExecProcedureResponse\022D\n\017IsP" + + "rocedureDone\022\027.IsProcedureDoneRequest\032\030.", + "IsProcedureDoneResponse\022D\n\017ModifyNamespa" + + "ce\022\027.ModifyNamespaceRequest\032\030.ModifyName" + + "spaceResponse\022D\n\017CreateNamespace\022\027.Creat" + + "eNamespaceRequest\032\030.CreateNamespaceRespo" + + "nse\022D\n\017DeleteNamespace\022\027.DeleteNamespace" + + "Request\032\030.DeleteNamespaceResponse\022Y\n\026Get" + + "NamespaceDescriptor\022\036.GetNamespaceDescri" + + "ptorRequest\032\037.GetNamespaceDescriptorResp" + + "onse\022_\n\030ListNamespaceDescriptors\022 .ListN" + + "amespaceDescriptorsRequest\032!.ListNamespa", + "ceDescriptorsResponse\022t\n\037ListTableDescri" + + "ptorsByNamespace\022\'.ListTableDescriptorsB" + + "yNamespaceRequest\032(.ListTableDescriptors" + + "ByNamespaceResponse\022b\n\031ListTableNamesByN" + + "amespace\022!.ListTableNamesByNamespaceRequ" + + "est\032\".ListTableNamesByNamespaceResponse\022" + + ">\n\rGetTableState\022\025.GetTableStateRequest\032" + + "\026.GetTableStateResponse\022/\n\010SetQuota\022\020.Se" + + "tQuotaRequest\032\021.SetQuotaResponse\022f\n\037getL" + + "astMajorCompactionTimestamp\022 .MajorCompa", + "ctionTimestampRequest\032!.MajorCompactionT" + + "imestampResponse\022x\n(getLastMajorCompacti" + + "onTimestampForRegion\022).MajorCompactionTi" + + "mestampForRegionRequest\032!.MajorCompactio" + + "nTimestampResponse\022M\n\022getProcedureResult" + + "\022\032.GetProcedureResultRequest\032\033.GetProced" + + "ureResultResponseBB\n*org.apache.hadoop.h" + + "base.protobuf.generatedB\014MasterProtosH\001\210" + + "\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -53550,7 +55180,7 @@ public final class MasterProtos { internal_static_AddColumnRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_AddColumnRequest_descriptor, - new java.lang.String[] { "TableName", "ColumnFamilies", }); + new java.lang.String[] { "TableName", "ColumnFamilies", "NonceGroup", "Nonce", }); internal_static_AddColumnResponse_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_AddColumnResponse_fieldAccessorTable = new @@ -53562,7 +55192,7 @@ public final class MasterProtos { internal_static_DeleteColumnRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DeleteColumnRequest_descriptor, - new java.lang.String[] { "TableName", "ColumnName", }); + new java.lang.String[] { "TableName", "ColumnName", "NonceGroup", "Nonce", }); internal_static_DeleteColumnResponse_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_DeleteColumnResponse_fieldAccessorTable = new @@ -53574,7 +55204,7 @@ public final class MasterProtos { internal_static_ModifyColumnRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ModifyColumnRequest_descriptor, - new java.lang.String[] { "TableName", "ColumnFamilies", }); + new java.lang.String[] { "TableName", "ColumnFamilies", "NonceGroup", "Nonce", }); internal_static_ModifyColumnResponse_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_ModifyColumnResponse_fieldAccessorTable = new @@ -53646,7 +55276,7 @@ public final class MasterProtos { internal_static_CreateTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CreateTableRequest_descriptor, - new java.lang.String[] { "TableSchema", "SplitKeys", }); + new java.lang.String[] { "TableSchema", "SplitKeys", "NonceGroup", "Nonce", }); internal_static_CreateTableResponse_descriptor = getDescriptor().getMessageTypes().get(17); internal_static_CreateTableResponse_fieldAccessorTable = new @@ -53658,7 +55288,7 @@ public final class MasterProtos { internal_static_DeleteTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DeleteTableRequest_descriptor, - new java.lang.String[] { "TableName", }); + new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); internal_static_DeleteTableResponse_descriptor = getDescriptor().getMessageTypes().get(19); internal_static_DeleteTableResponse_fieldAccessorTable = new @@ -53670,7 +55300,7 @@ public final class MasterProtos { internal_static_TruncateTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TruncateTableRequest_descriptor, - new java.lang.String[] { "TableName", "PreserveSplits", }); + new java.lang.String[] { "TableName", "PreserveSplits", "NonceGroup", "Nonce", }); internal_static_TruncateTableResponse_descriptor = getDescriptor().getMessageTypes().get(21); internal_static_TruncateTableResponse_fieldAccessorTable = new @@ -53682,7 +55312,7 @@ public final class MasterProtos { internal_static_EnableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableTableRequest_descriptor, - new java.lang.String[] { "TableName", }); + new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); internal_static_EnableTableResponse_descriptor = getDescriptor().getMessageTypes().get(23); internal_static_EnableTableResponse_fieldAccessorTable = new @@ -53694,7 +55324,7 @@ public final class MasterProtos { internal_static_DisableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DisableTableRequest_descriptor, - new java.lang.String[] { "TableName", }); + new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); internal_static_DisableTableResponse_descriptor = getDescriptor().getMessageTypes().get(25); internal_static_DisableTableResponse_fieldAccessorTable = new @@ -53706,7 +55336,7 @@ public final class MasterProtos { internal_static_ModifyTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ModifyTableRequest_descriptor, - new java.lang.String[] { "TableName", "TableSchema", }); + new java.lang.String[] { "TableName", "TableSchema", "NonceGroup", "Nonce", }); internal_static_ModifyTableResponse_descriptor = getDescriptor().getMessageTypes().get(27); internal_static_ModifyTableResponse_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java index 3c7dcdba324..d9d6ccb8dae 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java @@ -382,6 +382,34 @@ public final class ProcedureProtos { * */ com.google.protobuf.ByteString getStateData(); + + // optional uint64 nonce_group = 13 [default = 0]; + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+     * Nonce to prevent same procedure submit by multiple times
+     * 
+ */ + boolean hasNonceGroup(); + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+     * Nonce to prevent same procedure submit by multiple times
+     * 
+ */ + long getNonceGroup(); + + // optional uint64 nonce = 14 [default = 0]; + /** + * optional uint64 nonce = 14 [default = 0]; + */ + boolean hasNonce(); + /** + * optional uint64 nonce = 14 [default = 0]; + */ + long getNonce(); } /** * Protobuf type {@code Procedure} @@ -529,6 +557,16 @@ public final class ProcedureProtos { stateData_ = input.readBytes(); break; } + case 104: { + bitField0_ |= 0x00000800; + nonceGroup_ = input.readUInt64(); + break; + } + case 112: { + bitField0_ |= 0x00001000; + nonce_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -899,6 +937,46 @@ public final class ProcedureProtos { return stateData_; } + // optional uint64 nonce_group = 13 [default = 0]; + public static final int NONCE_GROUP_FIELD_NUMBER = 13; + private long nonceGroup_; + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+     * Nonce to prevent same procedure submit by multiple times
+     * 
+ */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+     * Nonce to prevent same procedure submit by multiple times
+     * 
+ */ + public long getNonceGroup() { + return nonceGroup_; + } + + // optional uint64 nonce = 14 [default = 0]; + public static final int NONCE_FIELD_NUMBER = 14; + private long nonce_; + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + private void initFields() { className_ = ""; parentId_ = 0L; @@ -912,6 +990,8 @@ public final class ProcedureProtos { exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance(); result_ = com.google.protobuf.ByteString.EMPTY; stateData_ = com.google.protobuf.ByteString.EMPTY; + nonceGroup_ = 0L; + nonce_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -981,6 +1061,12 @@ public final class ProcedureProtos { if (((bitField0_ & 0x00000400) == 0x00000400)) { output.writeBytes(12, stateData_); } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeUInt64(13, nonceGroup_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeUInt64(14, nonce_); + } getUnknownFields().writeTo(output); } @@ -1043,6 +1129,14 @@ public final class ProcedureProtos { size += com.google.protobuf.CodedOutputStream .computeBytesSize(12, stateData_); } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(13, nonceGroup_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(14, nonce_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1123,6 +1217,16 @@ public final class ProcedureProtos { result = result && getStateData() .equals(other.getStateData()); } + result = result && (hasNonceGroup() == other.hasNonceGroup()); + if (hasNonceGroup()) { + result = result && (getNonceGroup() + == other.getNonceGroup()); + } + result = result && (hasNonce() == other.hasNonce()); + if (hasNonce()) { + result = result && (getNonce() + == other.getNonce()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1184,6 +1288,14 @@ public final class ProcedureProtos { hash = (37 * hash) + STATE_DATA_FIELD_NUMBER; hash = (53 * hash) + getStateData().hashCode(); } + if (hasNonceGroup()) { + hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonceGroup()); + } + if (hasNonce()) { + hash = (37 * hash) + NONCE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getNonce()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -1327,6 +1439,10 @@ public final class ProcedureProtos { bitField0_ = (bitField0_ & ~0x00000400); stateData_ = com.google.protobuf.ByteString.EMPTY; bitField0_ = (bitField0_ & ~0x00000800); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00001000); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00002000); return this; } @@ -1408,6 +1524,14 @@ public final class ProcedureProtos { to_bitField0_ |= 0x00000400; } result.stateData_ = stateData_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000800; + } + result.nonceGroup_ = nonceGroup_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00001000; + } + result.nonce_ = nonce_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1471,6 +1595,12 @@ public final class ProcedureProtos { if (other.hasStateData()) { setStateData(other.getStateData()); } + if (other.hasNonceGroup()) { + setNonceGroup(other.getNonceGroup()); + } + if (other.hasNonce()) { + setNonce(other.getNonce()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2274,6 +2404,88 @@ public final class ProcedureProtos { return this; } + // optional uint64 nonce_group = 13 [default = 0]; + private long nonceGroup_ ; + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+       * Nonce to prevent same procedure submit by multiple times
+       * 
+ */ + public boolean hasNonceGroup() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+       * Nonce to prevent same procedure submit by multiple times
+       * 
+ */ + public long getNonceGroup() { + return nonceGroup_; + } + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+       * Nonce to prevent same procedure submit by multiple times
+       * 
+ */ + public Builder setNonceGroup(long value) { + bitField0_ |= 0x00001000; + nonceGroup_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce_group = 13 [default = 0]; + * + *
+       * Nonce to prevent same procedure submit by multiple times
+       * 
+ */ + public Builder clearNonceGroup() { + bitField0_ = (bitField0_ & ~0x00001000); + nonceGroup_ = 0L; + onChanged(); + return this; + } + + // optional uint64 nonce = 14 [default = 0]; + private long nonce_ ; + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public boolean hasNonce() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public long getNonce() { + return nonce_; + } + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public Builder setNonce(long value) { + bitField0_ |= 0x00002000; + nonce_ = value; + onChanged(); + return this; + } + /** + * optional uint64 nonce = 14 [default = 0]; + */ + public Builder clearNonce() { + bitField0_ = (bitField0_ & ~0x00002000); + nonce_ = 0L; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:Procedure) } @@ -7124,33 +7336,34 @@ public final class ProcedureProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\017Procedure.proto\032\023ErrorHandling.proto\"\217" + + "\n\017Procedure.proto\032\023ErrorHandling.proto\"\271" + "\002\n\tProcedure\022\022\n\nclass_name\030\001 \002(\t\022\021\n\tpare" + "nt_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\022\n\nstart_ti" + "me\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\036\n\005state\030\006 \002(\0162\017" + ".ProcedureState\022\020\n\010stack_id\030\007 \003(\r\022\023\n\013las" + "t_update\030\010 \002(\004\022\017\n\007timeout\030\t \001(\r\022+\n\texcep" + "tion\030\n \001(\0132\030.ForeignExceptionMessage\022\016\n\006" + - "result\030\013 \001(\014\022\022\n\nstate_data\030\014 \001(\014\"+\n\027Sequ" + - "entialProcedureData\022\020\n\010executed\030\001 \002(\010\"*\n" + - "\031StateMachineProcedureData\022\r\n\005state\030\001 \003(", - "\r\"X\n\022ProcedureWALHeader\022\017\n\007version\030\001 \002(\r" + - "\022\014\n\004type\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_pr" + - "oc_id\030\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007ve" + - "rsion\030\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\214\001\n\025Pro" + - "cedureStoreTracker\0220\n\004node\030\001 \003(\0132\".Proce" + - "dureStoreTracker.TrackerNode\032A\n\013TrackerN" + - "ode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004\022\017" + - "\n\007deleted\030\003 \003(\004\"\266\001\n\021ProcedureWALEntry\022%\n" + - "\004type\030\001 \002(\0162\027.ProcedureWALEntry.Type\022\035\n\t" + - "procedure\030\002 \003(\0132\n.Procedure\022\017\n\007proc_id\030\003", - " \001(\004\"J\n\004Type\022\007\n\003EOF\020\001\022\010\n\004INIT\020\002\022\n\n\006INSER" + - "T\020\003\022\n\n\006UPDATE\020\004\022\n\n\006DELETE\020\005\022\013\n\007COMPACT\020\006" + - "*p\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" + - "RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIMEO" + - "UT\020\004\022\016\n\nROLLEDBACK\020\005\022\014\n\010FINISHED\020\006BE\n*or" + - "g.apache.hadoop.hbase.protobuf.generated" + - "B\017ProcedureProtosH\001\210\001\001\240\001\001" + "result\030\013 \001(\014\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonc" + + "e_group\030\r \001(\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027Se" + + "quentialProcedureData\022\020\n\010executed\030\001 \002(\010\"", + "*\n\031StateMachineProcedureData\022\r\n\005state\030\001 " + + "\003(\r\"X\n\022ProcedureWALHeader\022\017\n\007version\030\001 \002" + + "(\r\022\014\n\004type\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_" + + "proc_id\030\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007" + + "version\030\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\214\001\n\025P" + + "rocedureStoreTracker\0220\n\004node\030\001 \003(\0132\".Pro" + + "cedureStoreTracker.TrackerNode\032A\n\013Tracke" + + "rNode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004" + + "\022\017\n\007deleted\030\003 \003(\004\"\266\001\n\021ProcedureWALEntry\022" + + "%\n\004type\030\001 \002(\0162\027.ProcedureWALEntry.Type\022\035", + "\n\tprocedure\030\002 \003(\0132\n.Procedure\022\017\n\007proc_id" + + "\030\003 \001(\004\"J\n\004Type\022\007\n\003EOF\020\001\022\010\n\004INIT\020\002\022\n\n\006INS" + + "ERT\020\003\022\n\n\006UPDATE\020\004\022\n\n\006DELETE\020\005\022\013\n\007COMPACT" + + "\020\006*p\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014" + + "\n\010RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIM" + + "EOUT\020\004\022\016\n\nROLLEDBACK\020\005\022\014\n\010FINISHED\020\006BE\n*" + + "org.apache.hadoop.hbase.protobuf.generat" + + "edB\017ProcedureProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -7162,7 +7375,7 @@ public final class ProcedureProtos { internal_static_Procedure_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Procedure_descriptor, - new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", }); + new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", }); internal_static_SequentialProcedureData_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_SequentialProcedureData_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto index 990eb5b99e2..69371fae54e 100644 --- a/hbase-protocol/src/main/protobuf/Master.proto +++ b/hbase-protocol/src/main/protobuf/Master.proto @@ -36,6 +36,8 @@ import "Quota.proto"; message AddColumnRequest { required TableName table_name = 1; required ColumnFamilySchema column_families = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message AddColumnResponse { @@ -44,6 +46,8 @@ message AddColumnResponse { message DeleteColumnRequest { required TableName table_name = 1; required bytes column_name = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message DeleteColumnResponse { @@ -52,6 +56,8 @@ message DeleteColumnResponse { message ModifyColumnRequest { required TableName table_name = 1; required ColumnFamilySchema column_families = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message ModifyColumnResponse { @@ -106,6 +112,8 @@ message OfflineRegionResponse { message CreateTableRequest { required TableSchema table_schema = 1; repeated bytes split_keys = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message CreateTableResponse { @@ -114,6 +122,8 @@ message CreateTableResponse { message DeleteTableRequest { required TableName table_name = 1; + optional uint64 nonce_group = 2 [default = 0]; + optional uint64 nonce = 3 [default = 0]; } message DeleteTableResponse { @@ -123,6 +133,8 @@ message DeleteTableResponse { message TruncateTableRequest { required TableName tableName = 1; optional bool preserveSplits = 2 [default = false]; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message TruncateTableResponse { @@ -131,6 +143,8 @@ message TruncateTableResponse { message EnableTableRequest { required TableName table_name = 1; + optional uint64 nonce_group = 2 [default = 0]; + optional uint64 nonce = 3 [default = 0]; } message EnableTableResponse { @@ -139,6 +153,8 @@ message EnableTableResponse { message DisableTableRequest { required TableName table_name = 1; + optional uint64 nonce_group = 2 [default = 0]; + optional uint64 nonce = 3 [default = 0]; } message DisableTableResponse { @@ -148,6 +164,8 @@ message DisableTableResponse { message ModifyTableRequest { required TableName table_name = 1; required TableSchema table_schema = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; } message ModifyTableResponse { diff --git a/hbase-protocol/src/main/protobuf/Procedure.proto b/hbase-protocol/src/main/protobuf/Procedure.proto index 232c2903d4d..46ff9670d97 100644 --- a/hbase-protocol/src/main/protobuf/Procedure.proto +++ b/hbase-protocol/src/main/protobuf/Procedure.proto @@ -54,6 +54,10 @@ message Procedure { optional ForeignExceptionMessage exception = 10; optional bytes result = 11; // opaque (user) result structure optional bytes state_data = 12; // opaque (user) procedure internal-state + + // Nonce to prevent same procedure submit by multiple times + optional uint64 nonce_group = 13 [default = 0]; + optional uint64 nonce = 14 [default = 0]; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index ff5cb649177..bc52edb5597 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -353,7 +353,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024)); LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) + - ", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false)); + ", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false)); // Disable usage of meta replicas in the master this.conf.setBoolean(HConstants.USE_META_REPLICAS, false); @@ -1402,8 +1402,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long createTable(HTableDescriptor hTableDescriptor, - byte [][] splitKeys) throws IOException { + public long createTable( + final HTableDescriptor hTableDescriptor, + final byte [][] splitKeys, + final long nonceGroup, + final long nonce) throws IOException { if (isStopped()) { throw new MasterNotRunningException(); } @@ -1424,8 +1427,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // TableExistsException by saying if the schema is the same or not. ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(); long procId = this.procedureExecutor.submitProcedure( - new CreateTableProcedure(procedureExecutor.getEnvironment(), - hTableDescriptor, newRegions, latch)); + new CreateTableProcedure( + procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch), + nonceGroup, + nonce); latch.await(); if (cpHost != null) { @@ -1663,7 +1668,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long deleteTable(final TableName tableName) throws IOException { + public long deleteTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preDeleteTable(tableName); @@ -1673,7 +1681,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // TODO: We can handle/merge duplicate request ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(); long procId = this.procedureExecutor.submitProcedure( - new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch)); + new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch), + nonceGroup, + nonce); latch.await(); if (cpHost != null) { @@ -1684,7 +1694,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException { + public long truncateTable( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preTruncateTable(tableName); @@ -1692,7 +1706,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { LOG.info(getClientIdAuditPrefix() + " truncate " + tableName); long procId = this.procedureExecutor.submitProcedure( - new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits)); + new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits), + nonceGroup, + nonce); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1702,7 +1718,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor) + public void addColumn( + final TableName tableName, + final HColumnDescriptor columnDescriptor, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); checkCompression(columnDescriptor); @@ -1713,9 +1733,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } } // Execute the operation synchronously - wait for the operation to complete before continuing. - long procId = - this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor - .getEnvironment(), tableName, columnDescriptor)); + long procId = this.procedureExecutor.submitProcedure( + new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { cpHost.postAddColumn(tableName, columnDescriptor); @@ -1723,7 +1744,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void modifyColumn(TableName tableName, HColumnDescriptor descriptor) + public void modifyColumn( + final TableName tableName, + final HColumnDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); checkCompression(descriptor); @@ -1736,9 +1761,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { LOG.info(getClientIdAuditPrefix() + " modify " + descriptor); // Execute the operation synchronously - wait for the operation to complete before continuing. - long procId = - this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor - .getEnvironment(), tableName, descriptor)); + long procId = this.procedureExecutor.submitProcedure( + new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, descriptor), + nonceGroup, + nonce); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1747,7 +1773,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void deleteColumn(final TableName tableName, final byte[] columnName) + public void deleteColumn( + final TableName tableName, + final byte[] columnName, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { @@ -1758,9 +1788,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName)); // Execute the operation synchronously - wait for the operation to complete before continuing. - long procId = - this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor - .getEnvironment(), tableName, columnName)); + long procId = this.procedureExecutor.submitProcedure( + new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnName), + nonceGroup, + nonce); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1769,7 +1800,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long enableTable(final TableName tableName) throws IOException { + public long enableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preEnableTable(tableName); @@ -1778,9 +1812,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // Execute the operation asynchronously - client will check the progress of the operation final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch(); - long procId = - this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor - .getEnvironment(), tableName, false, prepareLatch)); + long procId = this.procedureExecutor.submitProcedure( + new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch), + nonceGroup, + nonce); // Before returning to client, we want to make sure that the table is prepared to be // enabled (the table is locked and the table state is set). // @@ -1795,7 +1830,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long disableTable(final TableName tableName) throws IOException { + public long disableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preDisableTable(tableName); @@ -1805,9 +1843,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // Execute the operation asynchronously - client will check the progress of the operation final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch(); // Execute the operation asynchronously - client will check the progress of the operation - long procId = - this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor - .getEnvironment(), tableName, false, prepareLatch)); + long procId = this.procedureExecutor.submitProcedure( + new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch), + nonceGroup, + nonce); // Before returning to client, we want to make sure that the table is prepared to be // enabled (the table is locked and the table state is set). // @@ -1857,7 +1896,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long modifyTable(final TableName tableName, final HTableDescriptor descriptor) + public long modifyTable( + final TableName tableName, + final HTableDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); sanityCheckTableDescriptor(descriptor); @@ -1869,7 +1912,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // Execute the operation synchronously - wait for the operation completes before continuing. long procId = this.procedureExecutor.submitProcedure( - new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor)); + new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor), + nonceGroup, + nonce); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); 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 50070eff67e..c828880c631 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 @@ -348,8 +348,11 @@ public class MasterRpcServices extends RSRpcServices public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req) throws ServiceException { try { - master.addColumn(ProtobufUtil.toTableName(req.getTableName()), - HColumnDescriptor.convert(req.getColumnFamilies())); + master.addColumn( + ProtobufUtil.toTableName(req.getTableName()), + HColumnDescriptor.convert(req.getColumnFamilies()), + req.getNonceGroup(), + req.getNonce()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -416,7 +419,8 @@ public class MasterRpcServices extends RSRpcServices HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema()); byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req); try { - long procId = master.createTable(hTableDescriptor, splitKeys); + long procId = + master.createTable(hTableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce()); return CreateTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -427,8 +431,11 @@ public class MasterRpcServices extends RSRpcServices public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req) throws ServiceException { try { - master.deleteColumn(ProtobufUtil.toTableName(req.getTableName()), - req.getColumnName().toByteArray()); + master.deleteColumn( + ProtobufUtil.toTableName(req.getTableName()), + req.getColumnName().toByteArray(), + req.getNonceGroup(), + req.getNonce()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -472,7 +479,8 @@ public class MasterRpcServices extends RSRpcServices public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request) throws ServiceException { try { - long procId = master.deleteTable(ProtobufUtil.toTableName(request.getTableName())); + long procId = master.deleteTable(ProtobufUtil.toTableName( + request.getTableName()), request.getNonceGroup(), request.getNonce()); return DeleteTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -483,9 +491,11 @@ public class MasterRpcServices extends RSRpcServices public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request) throws ServiceException { try { - long procId = - master.truncateTable(ProtobufUtil.toTableName(request.getTableName()), - request.getPreserveSplits()); + long procId = master.truncateTable( + ProtobufUtil.toTableName(request.getTableName()), + request.getPreserveSplits(), + request.getNonceGroup(), + request.getNonce()); return TruncateTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -496,7 +506,10 @@ public class MasterRpcServices extends RSRpcServices public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request) throws ServiceException { try { - long procId = master.disableTable(ProtobufUtil.toTableName(request.getTableName())); + long procId = master.disableTable( + ProtobufUtil.toTableName(request.getTableName()), + request.getNonceGroup(), + request.getNonce()); return DisableTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -582,7 +595,10 @@ public class MasterRpcServices extends RSRpcServices public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request) throws ServiceException { try { - long procId = master.enableTable(ProtobufUtil.toTableName(request.getTableName())); + long procId = master.enableTable( + ProtobufUtil.toTableName(request.getTableName()), + request.getNonceGroup(), + request.getNonce()); return EnableTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -1069,8 +1085,11 @@ public class MasterRpcServices extends RSRpcServices public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req) throws ServiceException { try { - master.modifyColumn(ProtobufUtil.toTableName(req.getTableName()), - HColumnDescriptor.convert(req.getColumnFamilies())); + master.modifyColumn( + ProtobufUtil.toTableName(req.getTableName()), + HColumnDescriptor.convert(req.getColumnFamilies()), + req.getNonceGroup(), + req.getNonce()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -1093,8 +1112,11 @@ public class MasterRpcServices extends RSRpcServices public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req) throws ServiceException { try { - master.modifyTable(ProtobufUtil.toTableName(req.getTableName()), - HTableDescriptor.convert(req.getTableSchema())); + master.modifyTable( + ProtobufUtil.toTableName(req.getTableName()), + HTableDescriptor.convert(req.getTableSchema()), + req.getNonceGroup(), + req.getNonce()); } catch (IOException ioe) { throw new ServiceException(ioe); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 70851875498..7d70dc15ff3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -103,75 +103,125 @@ public interface MasterServices extends Server { * Create a table using the given table definition. * @param desc The table definition * @param splitKeys Starting row keys for the initial table regions. If null + * @param nonceGroup + * @param nonce * a single region is created. */ - long createTable(HTableDescriptor desc, byte[][] splitKeys) - throws IOException; + long createTable( + final HTableDescriptor desc, + final byte[][] splitKeys, + final long nonceGroup, + final long nonce) throws IOException; /** * Delete a table * @param tableName The table name + * @param nonceGroup + * @param nonce * @throws IOException */ - long deleteTable(final TableName tableName) throws IOException; + long deleteTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException; /** * Truncate a table * @param tableName The table name * @param preserveSplits True if the splits should be preserved + * @param nonceGroup + * @param nonce * @throws IOException */ - public long truncateTable(final TableName tableName, boolean preserveSplits) throws IOException; + public long truncateTable( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) throws IOException; /** * Modify the descriptor of an existing table * @param tableName The table name * @param descriptor The updated table descriptor + * @param nonceGroup + * @param nonce * @throws IOException */ - long modifyTable(final TableName tableName, final HTableDescriptor descriptor) + long modifyTable( + final TableName tableName, + final HTableDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException; /** * Enable an existing table * @param tableName The table name + * @param nonceGroup + * @param nonce * @throws IOException */ - long enableTable(final TableName tableName) throws IOException; + long enableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException; /** * Disable an existing table * @param tableName The table name + * @param nonceGroup + * @param nonce * @throws IOException */ - long disableTable(final TableName tableName) throws IOException; + long disableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException; /** * Add a new column to an existing table * @param tableName The table name * @param column The column definition + * @param nonceGroup + * @param nonce * @throws IOException */ - void addColumn(final TableName tableName, final HColumnDescriptor column) + void addColumn( + final TableName tableName, + final HColumnDescriptor column, + final long nonceGroup, + final long nonce) throws IOException; /** * Modify the column descriptor of an existing column in an existing table * @param tableName The table name * @param descriptor The updated column definition + * @param nonceGroup + * @param nonce * @throws IOException */ - void modifyColumn(TableName tableName, HColumnDescriptor descriptor) + void modifyColumn( + final TableName tableName, + final HColumnDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException; /** * Delete a column from an existing table * @param tableName The table name * @param columnName The column name + * @param nonceGroup + * @param nonce * @throws IOException */ - void deleteColumn(final TableName tableName, final byte[] columnName) + void deleteColumn( + final TableName tableName, + final byte[] columnName, + final long nonceGroup, + final long nonce) throws IOException; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java index 0d974b1edd8..b2b656b1d81 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NonceKey; import com.google.common.annotations.VisibleForTesting; @@ -102,38 +103,6 @@ public class ServerNonceManager { } } - /** - * This implementation is not smart and just treats nonce group and nonce as random bits. - */ - // TODO: we could use pure byte arrays, but then we wouldn't be able to use hash map. - private static class NonceKey { - private long group; - private long nonce; - - public NonceKey(long group, long nonce) { - assert nonce != HConstants.NO_NONCE; - this.group = group; - this.nonce = nonce; - } - - @Override - public boolean equals(Object obj) { - if (obj == null || !(obj instanceof NonceKey)) return false; - NonceKey nk = ((NonceKey)obj); - return this.nonce == nk.nonce && this.group == nk.group; - } - - @Override - public int hashCode() { - return (int)((group >> 32) ^ group ^ (nonce >> 32) ^ nonce); - } - - @Override - public String toString() { - return "[" + group + ":" + nonce + "]"; - } - } - /** * Nonces. * Approximate overhead per nonce: 64 bytes from hashmap, 32 from two objects (k/v), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index 131ff14ddcc..4ccf4dc6aca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -136,7 +136,9 @@ public class AccessControlLists { // Set cache data blocks in L1 if more than one cache tier deployed; e.g. this will // be the case if we are using CombinedBlockCache (Bucket Cache). .setCacheDataInL1(true)), - null); + null, + HConstants.NO_NONCE, + HConstants.NO_NONCE); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index b588ec593e2..cb194b1795d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -198,7 +198,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements DisabledRegionSplitPolicy.class.getName()); labelsTable.setValue(Bytes.toBytes(HConstants.DISALLOW_WRITES_IN_RECOVERING), Bytes.toBytes(true)); - master.createTable(labelsTable, null); + master.createTable(labelsTable, null, HConstants.NO_NONCE, HConstants.NO_NONCE); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index fbca881fb1d..4b8de7a9953 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Matchers; @@ -67,6 +68,10 @@ public class TestHBaseAdminNoCluster { * @throws MasterNotRunningException * @throws ServiceException */ + //TODO: Clean up, with Procedure V2 and nonce to prevent the same procedure to call mulitple + // time, this test is invalid anymore. Just keep the test around for some time before + // fully removing it. + @Ignore @Test public void testMasterMonitorCallableRetries() throws MasterNotRunningException, ZooKeeperConnectionException, IOException, ServiceException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java index 264e62f7b18..51861d64b31 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -1056,7 +1056,7 @@ public class TestAssignmentManagerOnCluster { assertEquals(oldServerName, regionStates.getRegionServerOfRegion(hri)); // Disable the table now. - master.disableTable(hri.getTable()); + master.disableTable(hri.getTable(), HConstants.NO_NONCE, HConstants.NO_NONCE); // Kill the hosting server, which doesn't have meta on it. cluster.killRegionServer(oldServerName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index adc17c3ad47..bc9af45e581 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -226,8 +226,11 @@ public class TestCatalogJanitor { } @Override - public long createTable(HTableDescriptor desc, byte[][] splitKeys) - throws IOException { + public long createTable( + final HTableDescriptor desc, + final byte[][] splitKeys, + final long nonceGroup, + final long nonce) throws IOException { // no-op return -1; } @@ -427,43 +430,68 @@ public class TestCatalogJanitor { } @Override - public long deleteTable(TableName tableName) throws IOException { + public long deleteTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException { + public long truncateTable( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public long modifyTable(TableName tableName, HTableDescriptor descriptor) - throws IOException { + public long modifyTable( + final TableName tableName, + final HTableDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public long enableTable(TableName tableName) throws IOException { + public long enableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public long disableTable(TableName tableName) throws IOException { + public long disableTable( + TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public void addColumn(TableName tableName, HColumnDescriptor column) - throws IOException { } + public void addColumn( + final TableName tableName, + final HColumnDescriptor columnDescriptor, + final long nonceGroup, + final long nonce) throws IOException { } @Override - public void modifyColumn(TableName tableName, HColumnDescriptor descriptor) - throws IOException { } + public void modifyColumn( + final TableName tableName, + final HColumnDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { } @Override - public void deleteColumn(TableName tableName, byte[] columnName) - throws IOException { } + public void deleteColumn( + final TableName tableName, + final byte[] columnName, + final long nonceGroup, + final long nonce) throws IOException { } @Override public TableLockManager getTableLockManager() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java index c82aecc7489..394e339ce8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.TableDescriptor; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.NonceGenerator; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.TableState; @@ -426,6 +427,14 @@ public class MasterProcedureTestingUtility { return put; } + public static long generateNonceGroup(final HMaster master) { + return master.getConnection().getNonceGenerator().getNonceGroup(); + } + + public static long generateNonce(final HMaster master) { + return master.getConnection().getNonceGenerator().newNonce(); + } + public static class InjectAbortOnLoadListener implements ProcedureExecutor.ProcedureExecutorListener { private final ProcedureExecutor procExec; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java index 1490aa1148d..5d8da9c674c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; 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.InvalidFamilyOperationException; import org.apache.hadoop.hbase.TableName; @@ -47,6 +48,9 @@ public class TestAddColumnFamilyProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -69,6 +73,9 @@ public class TestAddColumnFamilyProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -92,9 +99,10 @@ public class TestAddColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f3"); // Test 1: Add a column family online - long procId1 = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor1)); + long procId1 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor1), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -104,9 +112,10 @@ public class TestAddColumnFamilyProcedure { // Test 2: Add a column family offline UTIL.getHBaseAdmin().disableTable(tableName); - long procId2 = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor2)); + long procId2 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2), + nonceGroup + 1, + nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -125,9 +134,10 @@ public class TestAddColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1"); // add the column family - long procId1 = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId1 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -135,9 +145,10 @@ public class TestAddColumnFamilyProcedure { tableName, cf2); // add the column family that exists - long procId2 = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId2 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup + 1, + nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); @@ -149,9 +160,10 @@ public class TestAddColumnFamilyProcedure { // Do the same add the existing column family - this time offline UTIL.getHBaseAdmin().disableTable(tableName); - long procId3 = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId3 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup + 2, + nonce + 2); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId3); @@ -162,6 +174,37 @@ public class TestAddColumnFamilyProcedure { assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException); } + @Test(timeout=60000) + public void testAddSameColumnFamilyTwiceWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testAddSameColumnFamilyTwiceWithSameNonce"); + final String cf2 = "cf2"; + final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2); + + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1"); + + // add the column family + long procId1 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); + long procId2 = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); + MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(), + tableName, cf2); + + // Wait the completion and expect not fail - because it is the same proc + ProcedureTestingUtility.waitProcedure(procExec, procId2); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); + assertTrue(procId1 == procId2); + } + @Test(timeout = 60000) public void testRecoveryAndDoubleExecutionOffline() throws Exception { final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline"); @@ -176,9 +219,10 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = AddColumnFamilyState.values().length; @@ -202,9 +246,10 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = AddColumnFamilyState.values().length; @@ -228,9 +273,10 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = - procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, - columnDescriptor)); + long procId = procExec.submitProcedure( + new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); int numberOfSteps = AddColumnFamilyState.values().length - 2; // failing in the middle of proc MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index 7cd64b671a5..0aad5fa81cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableExistsException; @@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ModifyRegionUtils; - import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -43,10 +43,7 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; @Category({MasterTests.class, MediumTests.class}) public class TestCreateTableProcedure { @@ -54,6 +51,9 @@ public class TestCreateTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -76,6 +76,9 @@ public class TestCreateTableProcedure { @Before public void setup() throws Exception { resetProcExecutorTestingKillFlag(); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -125,12 +128,14 @@ public class TestCreateTableProcedure { // create the table long procId1 = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); // create another with the same name ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch(); long procId2 = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2)); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2), + nonceGroup + 1, + nonce + 1); ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1)); @@ -139,6 +144,29 @@ public class TestCreateTableProcedure { latch2.await(); } + @Test(timeout=60000) + public void testCreateTwiceWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testCreateTwiceWithSameNonce"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f"); + final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null); + + // create the table + long procId1 = procExec.submitProcedure( + new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); + + // create another with the same name + long procId2 = procExec.submitProcedure( + new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); + + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1)); + + ProcedureTestingUtility.waitProcedure(procExec, procId2); + ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2)); + assertTrue(procId1 == procId2); + } + @Test(timeout=60000) public void testRecoveryAndDoubleExecution() throws Exception { final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution"); @@ -152,7 +180,7 @@ public class TestCreateTableProcedure { HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2"); HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys); long procId = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); // Restart the executor and execute the step twice // NOTE: the 6 (number of CreateTableState steps) is hardcoded, @@ -180,7 +208,7 @@ public class TestCreateTableProcedure { htd.setRegionReplication(3); HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys); long procId = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); // NOTE: the 4 (number of CreateTableState steps) is hardcoded, // so you have to look at this test at least once when you add a new step. @@ -210,7 +238,7 @@ public class TestCreateTableProcedure { HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2"); HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys); long procId = procExec.submitProcedure( - new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions)); + new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); // NOTE: the 4 (number of CreateTableState steps) is hardcoded, // so you have to look at this test at least once when you add a new step. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java index dcf194085cf..ad683543b3f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; @@ -47,6 +48,9 @@ public class TestDeleteColumnFamilyProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -69,6 +73,9 @@ public class TestDeleteColumnFamilyProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -90,9 +97,10 @@ public class TestDeleteColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, "f3"); // Test 1: delete the column family that exists online - long procId1 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf1.getBytes())); + long procId1 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf1.getBytes()), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -102,9 +110,10 @@ public class TestDeleteColumnFamilyProcedure { // Test 2: delete the column family that exists offline UTIL.getHBaseAdmin().disableTable(tableName); - long procId2 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf2.getBytes())); + long procId2 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -120,9 +129,10 @@ public class TestDeleteColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2); // delete the column family that exists - long procId1 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf2.getBytes())); + long procId1 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); // First delete should succeed @@ -132,9 +142,10 @@ public class TestDeleteColumnFamilyProcedure { tableName, cf2); // delete the column family that does not exist - long procId2 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf2.getBytes())); + long procId2 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup + 1, + nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); @@ -147,9 +158,10 @@ public class TestDeleteColumnFamilyProcedure { // Try again, this time with table disabled. UTIL.getHBaseAdmin().disableTable(tableName); - long procId3 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf2.getBytes())); + long procId3 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup + 2, + nonce + 2); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId3); // Expect fail with InvalidFamilyOperationException @@ -159,6 +171,37 @@ public class TestDeleteColumnFamilyProcedure { assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException); } + @Test(timeout=60000) + public void testDeleteColumnFamilyTwiceWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testDeleteColumnFamilyTwiceWithSameNonce"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + final String cf2 = "cf2"; + + MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2); + + // delete the column family that exists + long procId1 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup, + nonce); + long procId2 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), + nonceGroup, + nonce); + + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); + MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(), + tableName, cf2); + + // Wait the completion and expect not fail - because it is the same proc + ProcedureTestingUtility.waitProcedure(procExec, procId2); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); + assertTrue(procId1 == procId2); + } + @Test(timeout=60000) public void testDeleteNonExistingColumnFamily() throws Exception { final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily"); @@ -169,9 +212,10 @@ public class TestDeleteColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2"); // delete the column family that does not exist - long procId1 = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf3.getBytes())); + long procId1 = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf3.getBytes()), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -195,9 +239,10 @@ public class TestDeleteColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the Delete procedure && kill the executor - long procId = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf4.getBytes())); + long procId = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = DeleteColumnFamilyState.values().length; @@ -221,9 +266,10 @@ public class TestDeleteColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the Delete procedure && kill the executor - long procId = - procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), - tableName, cf5.getBytes())); + long procId = procExec.submitProcedure( + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = DeleteColumnFamilyState.values().length; @@ -249,7 +295,9 @@ public class TestDeleteColumnFamilyProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes())); + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()), + nonceGroup, + nonce); // Failing before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should trigger the rollback // NOTE: the 1 (number before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT step) is hardcoded, @@ -280,7 +328,9 @@ public class TestDeleteColumnFamilyProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes())); + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()), + nonceGroup, + nonce); // Failing after DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should not trigger the rollback. // NOTE: the 4 (number of DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT + 1 step) is hardcoded, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java index 6795b222b06..f4afd31c9be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTa import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -42,10 +42,7 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; @Category({MasterTests.class, MediumTests.class}) public class TestDeleteTableProcedure { @@ -53,6 +50,9 @@ public class TestDeleteTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private long nonceGroup = HConstants.NO_NONCE; + private long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -77,6 +77,10 @@ public class TestDeleteTableProcedure { final ProcedureExecutor procExec = getMasterProcedureExecutor(); ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false); assertTrue("expected executor to be running", procExec.isRunning()); + + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -123,10 +127,10 @@ public class TestDeleteTableProcedure { // delete the table (that exists) long procId1 = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName)); + new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); // delete the table (that will no longer exist) long procId2 = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName)); + new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup + 1, nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -144,6 +148,36 @@ public class TestDeleteTableProcedure { assertTrue(result.getException().getCause() instanceof TableNotFoundException); } + @Test(timeout=60000) + public void testDoubleDeletedTableWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testDoubleDeletedTableWithSameNonce"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + HRegionInfo[] regions = MasterProcedureTestingUtility.createTable( + procExec, tableName, null, "f"); + UTIL.getHBaseAdmin().disableTable(tableName); + + // delete the table (that exists) + long procId1 = procExec.submitProcedure( + new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); + // delete the table (that will no longer exist) + long procId2 = procExec.submitProcedure( + new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); + + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.waitProcedure(procExec, procId2); + + // First delete should succeed + ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); + MasterProcedureTestingUtility.validateTableDeletion( + UTIL.getHBaseCluster().getMaster(), tableName, regions, "f"); + + // Second delete should not fail, because it is the same delete + ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); + assertTrue(procId1 == procId2); + } + @Test(timeout=60000) public void testSimpleDelete() throws Exception { final TableName tableName = TableName.valueOf("testSimpleDelete"); @@ -190,7 +224,7 @@ public class TestDeleteTableProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName)); + new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); // Restart the executor and execute the step twice // NOTE: the 6 (number of DeleteTableState steps) is hardcoded, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java index 0537ccc3473..6959af9f128 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableT import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -49,6 +49,9 @@ public class TestDisableTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -71,6 +74,9 @@ public class TestDisableTableProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -91,7 +97,7 @@ public class TestDisableTableProcedure { // Disable the table long procId = procExec.submitProcedure( - new DisableTableProcedure(procExec.getEnvironment(), tableName, false)); + new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId); @@ -108,7 +114,7 @@ public class TestDisableTableProcedure { // Disable the table long procId1 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false)); + procExec.getEnvironment(), tableName, false), nonceGroup, nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -117,7 +123,7 @@ public class TestDisableTableProcedure { // Disable the table again - expect failure long procId2 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false)); + procExec.getEnvironment(), tableName, false), nonceGroup + 1, nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureResult result = procExec.getResult(procId2); @@ -130,7 +136,7 @@ public class TestDisableTableProcedure { final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch(); long procId3 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false, prepareLatch)); + procExec.getEnvironment(), tableName, false, prepareLatch), nonceGroup + 2, nonce + 2); prepareLatch.await(); Assert.fail("Disable should throw exception through latch."); } catch (TableNotEnabledException tnee) { @@ -148,6 +154,29 @@ public class TestDisableTableProcedure { tableName); } + @Test(timeout = 60000) + public void testDisableTableTwiceWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testDisableTableTwiceWithSameNonce"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2"); + + // Disable the table + long procId1 = procExec.submitProcedure(new DisableTableProcedure( + procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + long procId2 = procExec.submitProcedure(new DisableTableProcedure( + procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); + MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(), + tableName); + + ProcedureTestingUtility.waitProcedure(procExec, procId2); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); + assertTrue(procId1 == procId2); + } + @Test(timeout=60000) public void testRecoveryAndDoubleExecution() throws Exception { final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution"); @@ -161,9 +190,8 @@ public class TestDisableTableProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the Disable procedure && kill the executor - long procId = - procExec.submitProcedure(new DisableTableProcedure(procExec.getEnvironment(), tableName, - false)); + long procId = procExec.submitProcedure( + new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); // Restart the executor and execute the step twice int numberOfSteps = DisableTableState.values().length; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java index 12c78e865af..0204e528abd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTa import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -49,6 +49,9 @@ public class TestEnableTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -71,6 +74,9 @@ public class TestEnableTableProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -92,7 +98,7 @@ public class TestEnableTableProcedure { // Enable the table long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId); @@ -100,6 +106,29 @@ public class TestEnableTableProcedure { tableName); } + @Test(timeout = 60000) + public void testEnableTableTwiceWithSameNonce() throws Exception { + final TableName tableName = TableName.valueOf("testEnableTableTwiceWithSameNonce"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2"); + UTIL.getHBaseAdmin().disableTable(tableName); + + // Enable the table + long procId1 = procExec.submitProcedure( + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + long procId2 = procExec.submitProcedure( + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId1); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); + // The second proc should succeed too - because it is the same proc. + ProcedureTestingUtility.waitProcedure(procExec, procId2); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); + assertTrue(procId1 == procId2); + } + @Test(timeout=60000, expected=TableNotDisabledException.class) public void testEnableNonDisabledTable() throws Exception { final TableName tableName = TableName.valueOf("testEnableNonExistingTable"); @@ -109,7 +138,7 @@ public class TestEnableTableProcedure { // Enable the table - expect failure long procId1 = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureResult result = procExec.getResult(procId1); @@ -119,7 +148,9 @@ public class TestEnableTableProcedure { // Enable the table with skipping table state check flag (simulate recovery scenario) long procId2 = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, true)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, true), + nonceGroup + 1, + nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -127,7 +158,9 @@ public class TestEnableTableProcedure { // Enable the table - expect failure from ProcedurePrepareLatch final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch(); long procId3 = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch), + nonceGroup + 2, + nonce + 2); prepareLatch.await(); Assert.fail("Enable should throw exception through latch."); } @@ -147,7 +180,7 @@ public class TestEnableTableProcedure { // Start the Enable procedure && kill the executor long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); // Restart the executor and execute the step twice int numberOfSteps = EnableTableState.values().length; @@ -175,7 +208,7 @@ public class TestEnableTableProcedure { // Start the Enable procedure && kill the executor long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); int numberOfSteps = EnableTableState.values().length - 2; // failing in the middle of proc MasterProcedureTestingUtility.testRollbackAndDoubleExecution( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java index d29ea256422..3b409558c8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; 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.InvalidFamilyOperationException; import org.apache.hadoop.hbase.TableName; @@ -47,6 +48,9 @@ public class TestModifyColumnFamilyProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -69,6 +73,9 @@ public class TestModifyColumnFamilyProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -94,8 +101,10 @@ public class TestModifyColumnFamilyProcedure { // Test 1: modify the column family online columnDescriptor.setBlocksize(newBlockSize); - long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure( - procExec.getEnvironment(), tableName, columnDescriptor)); + long procId1 = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -105,9 +114,10 @@ public class TestModifyColumnFamilyProcedure { // Test 2: modify the column family offline UTIL.getHBaseAdmin().disableTable(tableName); columnDescriptor.setBlocksize(newBlockSize * 2); - long procId2 = - procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(), - tableName, columnDescriptor)); + long procId2 = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup + 1, + nonce + 1); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -129,8 +139,10 @@ public class TestModifyColumnFamilyProcedure { // Modify the column family that does not exist columnDescriptor.setBlocksize(newBlockSize); - long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure( - procExec.getEnvironment(), tableName, columnDescriptor)); + long procId1 = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -158,8 +170,10 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure( - procExec.getEnvironment(), tableName, columnDescriptor)); + long procId = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = ModifyColumnFamilyState.values().length; @@ -190,9 +204,10 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = - procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(), - tableName, columnDescriptor)); + long procId = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Restart the executor and execute the step twice int numberOfSteps = ModifyColumnFamilyState.values().length; @@ -220,8 +235,10 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure( - procExec.getEnvironment(), tableName, columnDescriptor)); + long procId = procExec.submitProcedure( + new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), + nonceGroup, + nonce); // Failing in the middle of proc int numberOfSteps = ModifyColumnFamilyState.values().length - 2; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java index af29338a735..ebe70d29a9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java @@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -48,6 +49,9 @@ public class TestModifyTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -70,6 +74,9 @@ public class TestModifyTableProcedure { @Before public void setup() throws Exception { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -224,8 +231,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = - procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); + long procId = procExec.submitProcedure( + new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); // Restart the executor and execute the step twice int numberOfSteps = ModifyTableState.values().length; @@ -266,8 +273,8 @@ public class TestModifyTableProcedure { htd.removeFamily(cf3.getBytes()); // Start the Modify procedure && kill the executor - long procId = - procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); + long procId = procExec.submitProcedure( + new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); // Restart the executor and execute the step twice int numberOfSteps = ModifyTableState.values().length; @@ -304,8 +311,8 @@ public class TestModifyTableProcedure { htd.addFamily(new HColumnDescriptor(familyName)); // Start the Modify procedure && kill the executor - long procId = - procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); + long procId = procExec.submitProcedure( + new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); // Restart the executor and rollback the step twice int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc @@ -341,8 +348,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = - procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); + long procId = procExec.submitProcedure( + new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); // Restart the executor and rollback the step twice int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc @@ -379,8 +386,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = - procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); + long procId = procExec.submitProcedure( + new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); // Failing after MODIFY_TABLE_DELETE_FS_LAYOUT we should not trigger the rollback. // NOTE: the 5 (number of MODIFY_TABLE_DELETE_FS_LAYOUT + 1 step) is hardcoded, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java index 58acbaedd41..e9f57462876 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.Truncate import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -43,9 +43,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; @Category({MasterTests.class, MediumTests.class}) public class TestTruncateTableProcedure { @@ -53,6 +51,9 @@ public class TestTruncateTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static long nonceGroup = HConstants.NO_NONCE; + private static long nonce = HConstants.NO_NONCE; + private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -77,6 +78,10 @@ public class TestTruncateTableProcedure { final ProcedureExecutor procExec = getMasterProcedureExecutor(); ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false); assertTrue("expected executor to be running", procExec.isRunning()); + + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -210,7 +215,9 @@ public class TestTruncateTableProcedure { // Start the Truncate procedure && kill the executor long procId = procExec.submitProcedure( - new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits)); + new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits), + nonceGroup, + nonce); // Restart the executor and execute the step twice // NOTE: the 7 (number of TruncateTableState steps) is hardcoded,