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 ab73f50c869..f446d96ca35 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 @@ -206,6 +206,8 @@ public class HBaseAdmin implements Admin { private RpcRetryingCallerFactory rpcCallerFactory; + private NonceGenerator ng; + /** * Constructor. * See {@link #HBaseAdmin(Connection connection)} @@ -262,6 +264,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 @@ -638,7 +642,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); } }); @@ -808,7 +813,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); } }); @@ -920,7 +926,7 @@ public class HBaseAdmin implements Admin { @Override public Void call(int callTimeout) throws ServiceException { TruncateTableRequest req = RequestConverter.buildTruncateTableRequest( - tableName, preserveSplits); + tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce()); master.truncateTable(null, req); return null; } @@ -1056,7 +1062,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); } }); @@ -1243,7 +1250,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); } }); @@ -1522,7 +1530,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column); + AddColumnRequest req = RequestConverter.buildAddColumnRequest( + tableName, column, ng.getNonceGroup(), ng.newNonce()); master.addColumn(null,req); return null; } @@ -1569,7 +1578,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName); + DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest( + tableName, columnName, ng.getNonceGroup(), ng.newNonce()); master.deleteColumn(null,req); return null; } @@ -1618,7 +1628,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor); + ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest( + tableName, descriptor, ng.getNonceGroup(), ng.newNonce()); master.modifyColumn(null,req); return null; } @@ -2453,7 +2464,8 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd); + ModifyTableRequest request = RequestConverter.buildModifyTableRequest( + tableName, htd, ng.getNonceGroup(), ng.newNonce()); master.modifyTable(null, request); return null; } 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 fdc40a5451b..e9e54ba1126 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 @@ -1061,10 +1061,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(); } @@ -1076,10 +1081,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(); } @@ -1091,10 +1101,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(); } @@ -1176,9 +1191,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(); } @@ -1189,11 +1209,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(); } @@ -1203,9 +1228,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(); } @@ -1215,9 +1245,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(); } @@ -1229,7 +1264,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) { @@ -1237,6 +1275,8 @@ public final class RequestConverter { builder.addSplitKeys(ByteStringer.wrap(splitKey)); } } + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); return builder.build(); } @@ -1249,10 +1289,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(); } @@ -1354,7 +1399,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 7735b63f435..145f7f3564e 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 @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; 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; @@ -76,6 +75,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 @@ -285,6 +287,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 efa6a18a54e..1a901280fcc 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) } @@ -11506,6 +12586,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} @@ -11571,6 +12671,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) { @@ -11633,8 +12743,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() { @@ -11659,6 +12803,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); } @@ -11672,6 +12822,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; @@ -11700,6 +12858,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; @@ -11717,6 +12885,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; @@ -11833,6 +13009,10 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -11869,6 +13049,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; @@ -11888,6 +13076,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; } @@ -12040,6 +13234,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) } @@ -12501,6 +13761,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} @@ -12566,6 +13846,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) { @@ -12628,8 +13918,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() { @@ -12654,6 +13978,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); } @@ -12667,6 +13997,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; @@ -12695,6 +14033,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; @@ -12712,6 +14060,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; @@ -12828,6 +14184,10 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -12864,6 +14224,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; @@ -12883,6 +14251,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; } @@ -13035,6 +14409,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) } @@ -13510,6 +14950,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} @@ -13588,6 +15048,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) { @@ -13672,9 +15142,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() { @@ -13710,6 +15214,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); } @@ -13727,6 +15237,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; @@ -13760,6 +15278,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; @@ -13781,6 +15309,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; @@ -13904,6 +15440,10 @@ public final class MasterProtos { tableSchemaBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); + nonceGroup_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + nonce_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -13948,6 +15488,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; @@ -13970,6 +15518,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; } @@ -14247,6 +15801,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) } @@ -51909,228 +53529,237 @@ 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\"\027\n\025TruncateTableResponse\"4\n\022E" + - "nableTableRequest\022\036\n\ntable_name\030\001 \002(\0132\n." + - "TableName\"&\n\023EnableTableResponse\022\017\n\007proc" + - "_id\030\001 \001(\004\"5\n\023DisableTableRequest\022\036\n\ntabl" + - "e_name\030\001 \002(\0132\n.TableName\"\'\n\024DisableTable" + - "Response\022\017\n\007proc_id\030\001 \001(\004\"X\n\022ModifyTable" + - "Request\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022" + - "\"\n\014table_schema\030\002 \002(\0132\014.TableSchema\"\025\n\023M", - "odifyTableResponse\"K\n\026CreateNamespaceReq" + - "uest\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.Name" + - "spaceDescriptor\"\031\n\027CreateNamespaceRespon" + - "se\"/\n\026DeleteNamespaceRequest\022\025\n\rnamespac" + - "eName\030\001 \002(\t\"\031\n\027DeleteNamespaceResponse\"K" + - "\n\026ModifyNamespaceRequest\0221\n\023namespaceDes" + - "criptor\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027M" + - "odifyNamespaceResponse\"6\n\035GetNamespaceDe" + - "scriptorRequest\022\025\n\rnamespaceName\030\001 \002(\t\"S" + - "\n\036GetNamespaceDescriptorResponse\0221\n\023name", - "spaceDescriptor\030\001 \002(\0132\024.NamespaceDescrip" + - "tor\"!\n\037ListNamespaceDescriptorsRequest\"U" + - "\n ListNamespaceDescriptorsResponse\0221\n\023na" + - "mespaceDescriptor\030\001 \003(\0132\024.NamespaceDescr" + - "iptor\"?\n&ListTableDescriptorsByNamespace" + - "Request\022\025\n\rnamespaceName\030\001 \002(\t\"L\n\'ListTa" + - "bleDescriptorsByNamespaceResponse\022!\n\013tab" + - "leSchema\030\001 \003(\0132\014.TableSchema\"9\n ListTabl" + - "eNamesByNamespaceRequest\022\025\n\rnamespaceNam" + - "e\030\001 \002(\t\"B\n!ListTableNamesByNamespaceResp", - "onse\022\035\n\ttableName\030\001 \003(\0132\n.TableName\"\021\n\017S" + - "hutdownRequest\"\022\n\020ShutdownResponse\"\023\n\021St" + - "opMasterRequest\"\024\n\022StopMasterResponse\"\020\n" + - "\016BalanceRequest\"\'\n\017BalanceResponse\022\024\n\014ba" + - "lancer_ran\030\001 \002(\010\"<\n\031SetBalancerRunningRe" + - "quest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8" + - "\n\032SetBalancerRunningResponse\022\032\n\022prev_bal" + - "ance_value\030\001 \001(\010\"\032\n\030IsBalancerEnabledReq" + - "uest\",\n\031IsBalancerEnabledResponse\022\017\n\007ena" + - "bled\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026R", - "unCatalogScanResponse\022\023\n\013scan_result\030\001 \001" + - "(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006ena" + - "ble\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespons" + - "e\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanito" + - "rEnabledRequest\"0\n\037IsCatalogJanitorEnabl" + - "edResponse\022\r\n\005value\030\001 \002(\010\"9\n\017SnapshotReq" + - "uest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescrip" + - "tion\",\n\020SnapshotResponse\022\030\n\020expected_tim" + - "eout\030\001 \002(\003\"\036\n\034GetCompletedSnapshotsReque" + - "st\"H\n\035GetCompletedSnapshotsResponse\022\'\n\ts", - "napshots\030\001 \003(\0132\024.SnapshotDescription\"?\n\025" + - "DeleteSnapshotRequest\022&\n\010snapshot\030\001 \002(\0132" + - "\024.SnapshotDescription\"\030\n\026DeleteSnapshotR" + - "esponse\"@\n\026RestoreSnapshotRequest\022&\n\010sna" + - "pshot\030\001 \002(\0132\024.SnapshotDescription\"\031\n\027Res" + - "toreSnapshotResponse\"?\n\025IsSnapshotDoneRe" + - "quest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescri" + - "ption\"U\n\026IsSnapshotDoneResponse\022\023\n\004done\030" + - "\001 \001(\010:\005false\022&\n\010snapshot\030\002 \001(\0132\024.Snapsho" + - "tDescription\"F\n\034IsRestoreSnapshotDoneReq", - "uest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescrip" + - "tion\"4\n\035IsRestoreSnapshotDoneResponse\022\023\n" + - "\004done\030\001 \001(\010:\005false\"=\n\033GetSchemaAlterStat" + - "usRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableNam" + - "e\"T\n\034GetSchemaAlterStatusResponse\022\035\n\025yet" + - "_to_update_regions\030\001 \001(\r\022\025\n\rtotal_region" + - "s\030\002 \001(\r\"\202\001\n\032GetTableDescriptorsRequest\022\037" + - "\n\013table_names\030\001 \003(\0132\n.TableName\022\r\n\005regex" + - "\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010:\005fals" + - "e\022\021\n\tnamespace\030\004 \001(\t\"A\n\033GetTableDescript", - "orsResponse\022\"\n\014table_schema\030\001 \003(\0132\014.Tabl" + - "eSchema\"[\n\024GetTableNamesRequest\022\r\n\005regex" + - "\030\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010:\005fals" + - "e\022\021\n\tnamespace\030\003 \001(\t\"8\n\025GetTableNamesRes" + - "ponse\022\037\n\013table_names\030\001 \003(\0132\n.TableName\"\031" + - "\n\027GetClusterStatusRequest\"B\n\030GetClusterS" + - "tatusResponse\022&\n\016cluster_status\030\001 \002(\0132\016." + - "ClusterStatus\"\030\n\026IsMasterRunningRequest\"" + - "4\n\027IsMasterRunningResponse\022\031\n\021is_master_" + - "running\030\001 \002(\010\"@\n\024ExecProcedureRequest\022(\n", - "\tprocedure\030\001 \002(\0132\025.ProcedureDescription\"" + - "F\n\025ExecProcedureResponse\022\030\n\020expected_tim" + - "eout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"B\n\026IsPro" + - "cedureDoneRequest\022(\n\tprocedure\030\001 \001(\0132\025.P" + - "rocedureDescription\"W\n\027IsProcedureDoneRe" + - "sponse\022\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snapshot\030" + - "\002 \001(\0132\025.ProcedureDescription\",\n\031GetProce" + - "dureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\347\001\n\032G" + - "etProcedureResultResponse\0220\n\005state\030\001 \002(\016" + - "2!.GetProcedureResultResponse.State\022\022\n\ns", - "tart_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006" + - "result\030\004 \001(\014\022+\n\texception\030\005 \001(\0132\030.Foreig" + - "nExceptionMessage\"1\n\005State\022\r\n\tNOT_FOUND\020" + - "\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"\273\001\n\017SetQuot" + - "aRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_grou" + - "p\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\036\n\ntable_name" + - "\030\004 \001(\0132\n.TableName\022\022\n\nremove_all\030\005 \001(\010\022\026" + - "\n\016bypass_globals\030\006 \001(\010\022\"\n\010throttle\030\007 \001(\013" + - "2\020.ThrottleRequest\"\022\n\020SetQuotaResponse\"A" + - "\n\037MajorCompactionTimestampRequest\022\036\n\ntab", - "le_name\030\001 \002(\0132\n.TableName\"L\n(MajorCompac" + - "tionTimestampForRegionRequest\022 \n\006region\030" + - "\001 \002(\0132\020.RegionSpecifier\"@\n MajorCompacti" + - "onTimestampResponse\022\034\n\024compaction_timest" + - "amp\030\001 \002(\0032\243\033\n\rMasterService\022S\n\024GetSchema" + - "AlterStatus\022\034.GetSchemaAlterStatusReques" + - "t\032\035.GetSchemaAlterStatusResponse\022P\n\023GetT" + - "ableDescriptors\022\033.GetTableDescriptorsReq" + - "uest\032\034.GetTableDescriptorsResponse\022>\n\rGe" + - "tTableNames\022\025.GetTableNamesRequest\032\026.Get", - "TableNamesResponse\022G\n\020GetClusterStatus\022\030" + - ".GetClusterStatusRequest\032\031.GetClusterSta" + - "tusResponse\022D\n\017IsMasterRunning\022\027.IsMaste" + - "rRunningRequest\032\030.IsMasterRunningRespons" + - "e\0222\n\tAddColumn\022\021.AddColumnRequest\032\022.AddC" + - "olumnResponse\022;\n\014DeleteColumn\022\024.DeleteCo" + - "lumnRequest\032\025.DeleteColumnResponse\022;\n\014Mo" + - "difyColumn\022\024.ModifyColumnRequest\032\025.Modif" + - "yColumnResponse\0225\n\nMoveRegion\022\022.MoveRegi" + - "onRequest\032\023.MoveRegionResponse\022Y\n\026Dispat", - "chMergingRegions\022\036.DispatchMergingRegion" + - "sRequest\032\037.DispatchMergingRegionsRespons" + - "e\022;\n\014AssignRegion\022\024.AssignRegionRequest\032" + - "\025.AssignRegionResponse\022A\n\016UnassignRegion" + - "\022\026.UnassignRegionRequest\032\027.UnassignRegio" + - "nResponse\022>\n\rOfflineRegion\022\025.OfflineRegi" + - "onRequest\032\026.OfflineRegionResponse\0228\n\013Del" + - "eteTable\022\023.DeleteTableRequest\032\024.DeleteTa" + - "bleResponse\022>\n\rtruncateTable\022\025.TruncateT" + - "ableRequest\032\026.TruncateTableResponse\0228\n\013E", - "nableTable\022\023.EnableTableRequest\032\024.Enable" + - "TableResponse\022;\n\014DisableTable\022\024.DisableT" + - "ableRequest\032\025.DisableTableResponse\0228\n\013Mo" + - "difyTable\022\023.ModifyTableRequest\032\024.ModifyT" + - "ableResponse\0228\n\013CreateTable\022\023.CreateTabl" + - "eRequest\032\024.CreateTableResponse\022/\n\010Shutdo" + - "wn\022\020.ShutdownRequest\032\021.ShutdownResponse\022" + - "5\n\nStopMaster\022\022.StopMasterRequest\032\023.Stop" + - "MasterResponse\022,\n\007Balance\022\017.BalanceReque" + - "st\032\020.BalanceResponse\022M\n\022SetBalancerRunni", - "ng\022\032.SetBalancerRunningRequest\032\033.SetBala" + - "ncerRunningResponse\022J\n\021IsBalancerEnabled" + - "\022\031.IsBalancerEnabledRequest\032\032.IsBalancer" + - "EnabledResponse\022A\n\016RunCatalogScan\022\026.RunC" + - "atalogScanRequest\032\027.RunCatalogScanRespon" + - "se\022S\n\024EnableCatalogJanitor\022\034.EnableCatal" + - "ogJanitorRequest\032\035.EnableCatalogJanitorR" + - "esponse\022\\\n\027IsCatalogJanitorEnabled\022\037.IsC" + - "atalogJanitorEnabledRequest\032 .IsCatalogJ" + - "anitorEnabledResponse\022L\n\021ExecMasterServi", - "ce\022\032.CoprocessorServiceRequest\032\033.Coproce" + - "ssorServiceResponse\022/\n\010Snapshot\022\020.Snapsh" + - "otRequest\032\021.SnapshotResponse\022V\n\025GetCompl" + - "etedSnapshots\022\035.GetCompletedSnapshotsReq" + - "uest\032\036.GetCompletedSnapshotsResponse\022A\n\016" + - "DeleteSnapshot\022\026.DeleteSnapshotRequest\032\027" + - ".DeleteSnapshotResponse\022A\n\016IsSnapshotDon" + - "e\022\026.IsSnapshotDoneRequest\032\027.IsSnapshotDo" + - "neResponse\022D\n\017RestoreSnapshot\022\027.RestoreS" + - "napshotRequest\032\030.RestoreSnapshotResponse", - "\022V\n\025IsRestoreSnapshotDone\022\035.IsRestoreSna" + - "pshotDoneRequest\032\036.IsRestoreSnapshotDone" + - "Response\022>\n\rExecProcedure\022\025.ExecProcedur" + - "eRequest\032\026.ExecProcedureResponse\022E\n\024Exec" + - "ProcedureWithRet\022\025.ExecProcedureRequest\032" + - "\026.ExecProcedureResponse\022D\n\017IsProcedureDo" + - "ne\022\027.IsProcedureDoneRequest\032\030.IsProcedur" + - "eDoneResponse\022D\n\017ModifyNamespace\022\027.Modif" + - "yNamespaceRequest\032\030.ModifyNamespaceRespo" + - "nse\022D\n\017CreateNamespace\022\027.CreateNamespace", - "Request\032\030.CreateNamespaceResponse\022D\n\017Del" + - "eteNamespace\022\027.DeleteNamespaceRequest\032\030." + - "DeleteNamespaceResponse\022Y\n\026GetNamespaceD" + - "escriptor\022\036.GetNamespaceDescriptorReques" + - "t\032\037.GetNamespaceDescriptorResponse\022_\n\030Li" + - "stNamespaceDescriptors\022 .ListNamespaceDe" + - "scriptorsRequest\032!.ListNamespaceDescript" + - "orsResponse\022t\n\037ListTableDescriptorsByNam" + - "espace\022\'.ListTableDescriptorsByNamespace" + - "Request\032(.ListTableDescriptorsByNamespac", - "eResponse\022b\n\031ListTableNamesByNamespace\022!" + - ".ListTableNamesByNamespaceRequest\032\".List" + - "TableNamesByNamespaceResponse\022/\n\010SetQuot" + - "a\022\020.SetQuotaRequest\032\021.SetQuotaResponse\022f" + - "\n\037getLastMajorCompactionTimestamp\022 .Majo" + - "rCompactionTimestampRequest\032!.MajorCompa" + - "ctionTimestampResponse\022x\n(getLastMajorCo" + - "mpactionTimestampForRegion\022).MajorCompac" + - "tionTimestampForRegionRequest\032!.MajorCom" + - "pactionTimestampResponse\022M\n\022getProcedure", - "Result\022\032.GetProcedureResultRequest\032\033.Get" + - "ProcedureResultResponseBB\n*org.apache.ha" + - "doop.hbase.protobuf.generatedB\014MasterPro" + - "tosH\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\"\027\n\025TruncateTabl" + + "eResponse\"^\n\022EnableTableRequest\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\023EnableTable" + + "Response\022\017\n\007proc_id\030\001 \001(\004\"_\n\023DisableTabl" + + "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\024DisableTableResponse\022\017\n\007proc_id\030\001 " + + "\001(\004\"\202\001\n\022ModifyTableRequest\022\036\n\ntable_name" + + "\030\001 \002(\0132\n.TableName\022\"\n\014table_schema\030\002 \002(\013" + + "2\014.TableSchema\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020" + + "\n\005nonce\030\004 \001(\004:\0010\"\025\n\023ModifyTableResponse\"", + "K\n\026CreateNamespaceRequest\0221\n\023namespaceDe" + + "scriptor\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027" + + "CreateNamespaceResponse\"/\n\026DeleteNamespa" + + "ceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"\031\n\027Dele" + + "teNamespaceResponse\"K\n\026ModifyNamespaceRe" + + "quest\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.Nam" + + "espaceDescriptor\"\031\n\027ModifyNamespaceRespo" + + "nse\"6\n\035GetNamespaceDescriptorRequest\022\025\n\r" + + "namespaceName\030\001 \002(\t\"S\n\036GetNamespaceDescr" + + "iptorResponse\0221\n\023namespaceDescriptor\030\001 \002", + "(\0132\024.NamespaceDescriptor\"!\n\037ListNamespac" + + "eDescriptorsRequest\"U\n ListNamespaceDesc" + + "riptorsResponse\0221\n\023namespaceDescriptor\030\001" + + " \003(\0132\024.NamespaceDescriptor\"?\n&ListTableD" + + "escriptorsByNamespaceRequest\022\025\n\rnamespac" + + "eName\030\001 \002(\t\"L\n\'ListTableDescriptorsByNam" + + "espaceResponse\022!\n\013tableSchema\030\001 \003(\0132\014.Ta" + + "bleSchema\"9\n ListTableNamesByNamespaceRe" + + "quest\022\025\n\rnamespaceName\030\001 \002(\t\"B\n!ListTabl" + + "eNamesByNamespaceResponse\022\035\n\ttableName\030\001", + " \003(\0132\n.TableName\"\021\n\017ShutdownRequest\"\022\n\020S" + + "hutdownResponse\"\023\n\021StopMasterRequest\"\024\n\022" + + "StopMasterResponse\"\020\n\016BalanceRequest\"\'\n\017" + + "BalanceResponse\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\032SetBalancerRunnin" + + "gResponse\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n" + + "\030IsBalancerEnabledRequest\",\n\031IsBalancerE" + + "nabledResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCa" + + "talogScanRequest\"-\n\026RunCatalogScanRespon", + "se\022\023\n\013scan_result\030\001 \001(\005\"-\n\033EnableCatalog" + + "JanitorRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034Enable" + + "CatalogJanitorResponse\022\022\n\nprev_value\030\001 \001" + + "(\010\" \n\036IsCatalogJanitorEnabledRequest\"0\n\037" + + "IsCatalogJanitorEnabledResponse\022\r\n\005value" + + "\030\001 \002(\010\"9\n\017SnapshotRequest\022&\n\010snapshot\030\001 " + + "\002(\0132\024.SnapshotDescription\",\n\020SnapshotRes" + + "ponse\022\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034GetCo" + + "mpletedSnapshotsRequest\"H\n\035GetCompletedS" + + "napshotsResponse\022\'\n\tsnapshots\030\001 \003(\0132\024.Sn", + "apshotDescription\"?\n\025DeleteSnapshotReque" + + "st\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescripti" + + "on\"\030\n\026DeleteSnapshotResponse\"@\n\026RestoreS" + + "napshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Snaps" + + "hotDescription\"\031\n\027RestoreSnapshotRespons" + + "e\"?\n\025IsSnapshotDoneRequest\022&\n\010snapshot\030\001" + + " \001(\0132\024.SnapshotDescription\"U\n\026IsSnapshot" + + "DoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010sna" + + "pshot\030\002 \001(\0132\024.SnapshotDescription\"F\n\034IsR" + + "estoreSnapshotDoneRequest\022&\n\010snapshot\030\001 ", + "\001(\0132\024.SnapshotDescription\"4\n\035IsRestoreSn" + + "apshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\"" + + "=\n\033GetSchemaAlterStatusRequest\022\036\n\ntable_" + + "name\030\001 \002(\0132\n.TableName\"T\n\034GetSchemaAlter" + + "StatusResponse\022\035\n\025yet_to_update_regions\030" + + "\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"\202\001\n\032GetTabl" + + "eDescriptorsRequest\022\037\n\013table_names\030\001 \003(\013" + + "2\n.TableName\022\r\n\005regex\030\002 \001(\t\022!\n\022include_s" + + "ys_tables\030\003 \001(\010:\005false\022\021\n\tnamespace\030\004 \001(" + + "\t\"A\n\033GetTableDescriptorsResponse\022\"\n\014tabl", + "e_schema\030\001 \003(\0132\014.TableSchema\"[\n\024GetTable" + + "NamesRequest\022\r\n\005regex\030\001 \001(\t\022!\n\022include_s" + + "ys_tables\030\002 \001(\010:\005false\022\021\n\tnamespace\030\003 \001(" + + "\t\"8\n\025GetTableNamesResponse\022\037\n\013table_name" + + "s\030\001 \003(\0132\n.TableName\"\031\n\027GetClusterStatusR" + + "equest\"B\n\030GetClusterStatusResponse\022&\n\016cl" + + "uster_status\030\001 \002(\0132\016.ClusterStatus\"\030\n\026Is" + + "MasterRunningRequest\"4\n\027IsMasterRunningR" + + "esponse\022\031\n\021is_master_running\030\001 \002(\010\"@\n\024Ex" + + "ecProcedureRequest\022(\n\tprocedure\030\001 \002(\0132\025.", + "ProcedureDescription\"F\n\025ExecProcedureRes" + + "ponse\022\030\n\020expected_timeout\030\001 \001(\003\022\023\n\013retur" + + "n_data\030\002 \001(\014\"B\n\026IsProcedureDoneRequest\022(" + + "\n\tprocedure\030\001 \001(\0132\025.ProcedureDescription" + + "\"W\n\027IsProcedureDoneResponse\022\023\n\004done\030\001 \001(" + + "\010:\005false\022\'\n\010snapshot\030\002 \001(\0132\025.ProcedureDe" + + "scription\",\n\031GetProcedureResultRequest\022\017" + + "\n\007proc_id\030\001 \002(\004\"\347\001\n\032GetProcedureResultRe" + + "sponse\0220\n\005state\030\001 \002(\0162!.GetProcedureResu" + + "ltResponse.State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013", + "last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\022+\n\texc" + + "eption\030\005 \001(\0132\030.ForeignExceptionMessage\"1" + + "\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010F" + + "INISHED\020\002\"\273\001\n\017SetQuotaRequest\022\021\n\tuser_na" + + "me\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamespac" + + "e\030\003 \001(\t\022\036\n\ntable_name\030\004 \001(\0132\n.TableName\022" + + "\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 " + + "\001(\010\022\"\n\010throttle\030\007 \001(\0132\020.ThrottleRequest\"" + + "\022\n\020SetQuotaResponse\"A\n\037MajorCompactionTi" + + "mestampRequest\022\036\n\ntable_name\030\001 \002(\0132\n.Tab", + "leName\"L\n(MajorCompactionTimestampForReg" + + "ionRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpeci" + + "fier\"@\n MajorCompactionTimestampResponse" + + "\022\034\n\024compaction_timestamp\030\001 \002(\0032\243\033\n\rMaste" + + "rService\022S\n\024GetSchemaAlterStatus\022\034.GetSc" + + "hemaAlterStatusRequest\032\035.GetSchemaAlterS" + + "tatusResponse\022P\n\023GetTableDescriptors\022\033.G" + + "etTableDescriptorsRequest\032\034.GetTableDesc" + + "riptorsResponse\022>\n\rGetTableNames\022\025.GetTa" + + "bleNamesRequest\032\026.GetTableNamesResponse\022", + "G\n\020GetClusterStatus\022\030.GetClusterStatusRe" + + "quest\032\031.GetClusterStatusResponse\022D\n\017IsMa" + + "sterRunning\022\027.IsMasterRunningRequest\032\030.I" + + "sMasterRunningResponse\0222\n\tAddColumn\022\021.Ad" + + "dColumnRequest\032\022.AddColumnResponse\022;\n\014De" + + "leteColumn\022\024.DeleteColumnRequest\032\025.Delet" + + "eColumnResponse\022;\n\014ModifyColumn\022\024.Modify" + + "ColumnRequest\032\025.ModifyColumnResponse\0225\n\n" + + "MoveRegion\022\022.MoveRegionRequest\032\023.MoveReg" + + "ionResponse\022Y\n\026DispatchMergingRegions\022\036.", + "DispatchMergingRegionsRequest\032\037.Dispatch" + + "MergingRegionsResponse\022;\n\014AssignRegion\022\024" + + ".AssignRegionRequest\032\025.AssignRegionRespo" + + "nse\022A\n\016UnassignRegion\022\026.UnassignRegionRe" + + "quest\032\027.UnassignRegionResponse\022>\n\rOfflin" + + "eRegion\022\025.OfflineRegionRequest\032\026.Offline" + + "RegionResponse\0228\n\013DeleteTable\022\023.DeleteTa" + + "bleRequest\032\024.DeleteTableResponse\022>\n\rtrun" + + "cateTable\022\025.TruncateTableRequest\032\026.Trunc" + + "ateTableResponse\0228\n\013EnableTable\022\023.Enable", + "TableRequest\032\024.EnableTableResponse\022;\n\014Di" + + "sableTable\022\024.DisableTableRequest\032\025.Disab" + + "leTableResponse\0228\n\013ModifyTable\022\023.ModifyT" + + "ableRequest\032\024.ModifyTableResponse\0228\n\013Cre" + + "ateTable\022\023.CreateTableRequest\032\024.CreateTa" + + "bleResponse\022/\n\010Shutdown\022\020.ShutdownReques" + + "t\032\021.ShutdownResponse\0225\n\nStopMaster\022\022.Sto" + + "pMasterRequest\032\023.StopMasterResponse\022,\n\007B" + + "alance\022\017.BalanceRequest\032\020.BalanceRespons" + + "e\022M\n\022SetBalancerRunning\022\032.SetBalancerRun", + "ningRequest\032\033.SetBalancerRunningResponse" + + "\022J\n\021IsBalancerEnabled\022\031.IsBalancerEnable" + + "dRequest\032\032.IsBalancerEnabledResponse\022A\n\016" + + "RunCatalogScan\022\026.RunCatalogScanRequest\032\027" + + ".RunCatalogScanResponse\022S\n\024EnableCatalog" + + "Janitor\022\034.EnableCatalogJanitorRequest\032\035." + + "EnableCatalogJanitorResponse\022\\\n\027IsCatalo" + + "gJanitorEnabled\022\037.IsCatalogJanitorEnable" + + "dRequest\032 .IsCatalogJanitorEnabledRespon" + + "se\022L\n\021ExecMasterService\022\032.CoprocessorSer", + "viceRequest\032\033.CoprocessorServiceResponse" + + "\022/\n\010Snapshot\022\020.SnapshotRequest\032\021.Snapsho" + + "tResponse\022V\n\025GetCompletedSnapshots\022\035.Get" + + "CompletedSnapshotsRequest\032\036.GetCompleted" + + "SnapshotsResponse\022A\n\016DeleteSnapshot\022\026.De" + + "leteSnapshotRequest\032\027.DeleteSnapshotResp" + + "onse\022A\n\016IsSnapshotDone\022\026.IsSnapshotDoneR" + + "equest\032\027.IsSnapshotDoneResponse\022D\n\017Resto" + + "reSnapshot\022\027.RestoreSnapshotRequest\032\030.Re" + + "storeSnapshotResponse\022V\n\025IsRestoreSnapsh", + "otDone\022\035.IsRestoreSnapshotDoneRequest\032\036." + + "IsRestoreSnapshotDoneResponse\022>\n\rExecPro" + + "cedure\022\025.ExecProcedureRequest\032\026.ExecProc" + + "edureResponse\022E\n\024ExecProcedureWithRet\022\025." + + "ExecProcedureRequest\032\026.ExecProcedureResp" + + "onse\022D\n\017IsProcedureDone\022\027.IsProcedureDon" + + "eRequest\032\030.IsProcedureDoneResponse\022D\n\017Mo" + + "difyNamespace\022\027.ModifyNamespaceRequest\032\030" + + ".ModifyNamespaceResponse\022D\n\017CreateNamesp" + + "ace\022\027.CreateNamespaceRequest\032\030.CreateNam", + "espaceResponse\022D\n\017DeleteNamespace\022\027.Dele" + + "teNamespaceRequest\032\030.DeleteNamespaceResp" + + "onse\022Y\n\026GetNamespaceDescriptor\022\036.GetName" + + "spaceDescriptorRequest\032\037.GetNamespaceDes" + + "criptorResponse\022_\n\030ListNamespaceDescript" + + "ors\022 .ListNamespaceDescriptorsRequest\032!." + + "ListNamespaceDescriptorsResponse\022t\n\037List" + + "TableDescriptorsByNamespace\022\'.ListTableD" + + "escriptorsByNamespaceRequest\032(.ListTable" + + "DescriptorsByNamespaceResponse\022b\n\031ListTa", + "bleNamesByNamespace\022!.ListTableNamesByNa" + + "mespaceRequest\032\".ListTableNamesByNamespa" + + "ceResponse\022/\n\010SetQuota\022\020.SetQuotaRequest" + + "\032\021.SetQuotaResponse\022f\n\037getLastMajorCompa" + + "ctionTimestamp\022 .MajorCompactionTimestam" + + "pRequest\032!.MajorCompactionTimestampRespo" + + "nse\022x\n(getLastMajorCompactionTimestampFo" + + "rRegion\022).MajorCompactionTimestampForReg" + + "ionRequest\032!.MajorCompactionTimestampRes" + + "ponse\022M\n\022getProcedureResult\022\032.GetProcedu", + "reResultRequest\032\033.GetProcedureResultResp" + + "onseBB\n*org.apache.hadoop.hbase.protobuf" + + ".generatedB\014MasterProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -52142,7 +53771,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 @@ -52154,7 +53783,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 @@ -52166,7 +53795,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 @@ -52238,7 +53867,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 @@ -52250,7 +53879,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 @@ -52262,7 +53891,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 @@ -52274,7 +53903,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 @@ -52286,7 +53915,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 @@ -52298,7 +53927,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 0b2e875a3e5..469c0a40dd1 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 { @@ -130,6 +142,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 { @@ -138,6 +152,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 { @@ -147,6 +163,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 3e5d9094df2..bb2470c868b 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 @@ -305,7 +305,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { SnapshotManager snapshotManager; // monitor for distributed procedures MasterProcedureManagerHost mpmHost; - + // it is assigned after 'initialized' guard set to true, so should be volatile private volatile MasterQuotaManager quotaManager; @@ -355,7 +355,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); @@ -1435,8 +1435,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(); } @@ -1457,8 +1460,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) { @@ -1696,7 +1701,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); @@ -1706,7 +1714,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) { @@ -1717,7 +1727,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { + public void truncateTable( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preTruncateTable(tableName); @@ -1725,7 +1739,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) { @@ -1734,7 +1750,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); @@ -1745,9 +1765,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); @@ -1755,7 +1776,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); @@ -1768,9 +1793,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) { @@ -1779,7 +1805,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) { @@ -1790,9 +1820,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) { @@ -1801,7 +1832,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); @@ -1810,9 +1844,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). // @@ -1827,7 +1862,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); @@ -1837,9 +1875,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). // @@ -1890,7 +1929,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void modifyTable(final TableName tableName, final HTableDescriptor descriptor) + public void modifyTable( + final TableName tableName, + final HTableDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { checkInitialized(); sanityCheckTableDescriptor(descriptor); @@ -1902,7 +1945,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); @@ -2057,7 +2102,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { public MasterCoprocessorHost getMasterCoprocessorHost() { return cpHost; } - + @Override public MasterQuotaManager getMasterQuotaManager() { return quotaManager; 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 27fab4f2c04..fb4509097f6 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 @@ -347,8 +347,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); } @@ -415,7 +418,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); @@ -426,8 +430,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); } @@ -471,7 +478,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); @@ -482,8 +490,11 @@ public class MasterRpcServices extends RSRpcServices public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request) throws ServiceException { try { - master.truncateTable(ProtobufUtil.toTableName(request.getTableName()), - request.getPreserveSplits()); + master.truncateTable( + ProtobufUtil.toTableName(request.getTableName()), + request.getPreserveSplits(), + request.getNonceGroup(), + request.getNonce()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -494,7 +505,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); @@ -580,7 +594,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); @@ -1050,8 +1067,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); } @@ -1074,8 +1094,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 dd64bc84cd7..c22f0ed3dfc 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 @@ -98,75 +98,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 void truncateTable(final TableName tableName, boolean preserveSplits) throws IOException; + public void 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 */ - void modifyTable(final TableName tableName, final HTableDescriptor descriptor) + void 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 442f7eabacf..7d3b82e71ae 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 6672eb247e5..19774d74fa8 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 @@ -45,6 +45,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; @@ -66,6 +67,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 29766b3315f..343cd4cf61b 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 @@ -1154,7 +1154,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 053dc994fe2..fbdf6a2e457 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 @@ -224,8 +224,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; } @@ -254,7 +257,7 @@ public class TestCatalogJanitor { public MasterCoprocessorHost getMasterCoprocessorHost() { return null; } - + @Override public MasterQuotaManager getMasterQuotaManager() { return null; @@ -409,39 +412,64 @@ 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; + } + + public void truncateTable( + final TableName tableName, + final boolean preserveSplits, + final long nonceGroup, + final long nonce) throws IOException { + } + + @Override + public void modifyTable( + final TableName tableName, + final HTableDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { + } + + @Override + public long enableTable( + final TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { } - - - @Override - public void modifyTable(TableName tableName, HTableDescriptor descriptor) - throws IOException { } - - @Override - public long enableTable(TableName tableName) throws IOException { + public long disableTable( + TableName tableName, + final long nonceGroup, + final long nonce) throws IOException { return -1; } @Override - public long disableTable(TableName tableName) throws IOException { - return -1; - } + public void addColumn( + final TableName tableName, + final HColumnDescriptor columnDescriptor, + final long nonceGroup, + final long nonce) throws IOException { } @Override - public void addColumn(TableName tableName, HColumnDescriptor column) - throws IOException { } + public void modifyColumn( + final TableName tableName, + final HColumnDescriptor descriptor, + final long nonceGroup, + final long nonce) throws IOException { } @Override - public void modifyColumn(TableName tableName, HColumnDescriptor descriptor) - 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 00f82f4ce8a..77ca96e445f 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.TableStateManager; 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.MetaScanner; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; @@ -425,6 +426,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 05eb6027083..e09c8386496 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; @@ -46,6 +47,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); } @@ -68,6 +72,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 @@ -91,9 +98,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); @@ -103,9 +111,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); @@ -124,9 +133,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); @@ -134,9 +144,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); @@ -148,9 +159,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); @@ -161,6 +173,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"); @@ -175,9 +218,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; @@ -201,9 +245,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; @@ -227,9 +272,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 21129f27b27..73843e0e79d 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; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTa 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; @@ -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(MediumTests.class) public class TestCreateTableProcedure { @@ -53,6 +50,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); } @@ -75,6 +75,9 @@ public class TestCreateTableProcedure { @Before public void setup() throws Exception { resetProcExecutorTestingKillFlag(); + nonceGroup = + MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); + nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -124,12 +127,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)); @@ -138,6 +143,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"); @@ -151,7 +179,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, @@ -179,7 +207,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. @@ -209,7 +237,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 5aec002a0f8..e7efc2b5b29 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; @@ -46,6 +47,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); } @@ -68,6 +72,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 @@ -89,9 +96,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); @@ -101,9 +109,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); @@ -119,9 +128,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 @@ -131,9 +141,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); @@ -146,9 +157,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 @@ -158,6 +170,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"); @@ -168,9 +211,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); @@ -194,9 +238,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; @@ -220,9 +265,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; @@ -248,7 +294,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, @@ -279,7 +327,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 dc1d90a3c1c..a215c21148c 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; @@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState; 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; @@ -41,10 +41,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(MediumTests.class) public class TestDeleteTableProcedure { @@ -52,6 +49,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); } @@ -76,6 +76,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 @@ -122,10 +126,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); @@ -143,6 +147,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"); @@ -189,7 +223,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 d588a3b76a2..7f49e11bbcb 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; @@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState; 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; @@ -48,6 +48,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); } @@ -70,6 +73,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 @@ -90,7 +96,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); @@ -107,7 +113,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); @@ -116,7 +122,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); @@ -129,7 +135,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) { @@ -147,6 +153,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"); @@ -160,9 +189,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 8964adcf84a..7f8687abd45 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; @@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState; 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; @@ -48,6 +48,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); } @@ -70,6 +73,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 @@ -91,7 +97,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); @@ -99,6 +105,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"); @@ -108,7 +137,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); @@ -118,7 +147,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); @@ -126,7 +157,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."); } @@ -146,7 +179,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; @@ -174,7 +207,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 ea8882f1bf4..c7f05f1f8d7 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; @@ -46,6 +47,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); } @@ -68,6 +72,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 @@ -93,8 +100,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); @@ -104,9 +113,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); @@ -128,8 +138,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); @@ -157,8 +169,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; @@ -189,9 +203,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; @@ -219,8 +234,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 ebc238f8812..59229d47de3 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; @@ -47,6 +48,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); } @@ -69,6 +73,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 @@ -223,8 +230,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; @@ -265,8 +272,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; @@ -303,8 +310,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 @@ -340,8 +347,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 @@ -378,8 +385,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 1646ba0b944..2500ccae260 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; @@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState; 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,9 +42,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(MediumTests.class) public class TestTruncateTableProcedure { @@ -52,6 +50,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); } @@ -76,6 +77,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 @@ -209,7 +214,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,