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 f446d96ca35..ab73f50c869 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,8 +206,6 @@ public class HBaseAdmin implements Admin { private RpcRetryingCallerFactory rpcCallerFactory; - private NonceGenerator ng; - /** * Constructor. * See {@link #HBaseAdmin(Connection connection)} @@ -264,8 +262,6 @@ 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 @@ -642,8 +638,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public CreateTableResponse call(int callTimeout) throws ServiceException { - CreateTableRequest request = RequestConverter.buildCreateTableRequest( - desc, splitKeys, ng.getNonceGroup(), ng.newNonce()); + CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys); return master.createTable(null, request); } }); @@ -813,8 +808,7 @@ public class HBaseAdmin implements Admin { new MasterCallable(getConnection()) { @Override public DeleteTableResponse call(int callTimeout) throws ServiceException { - DeleteTableRequest req = - RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); + DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName); return master.deleteTable(null,req); } }); @@ -926,7 +920,7 @@ public class HBaseAdmin implements Admin { @Override public Void call(int callTimeout) throws ServiceException { TruncateTableRequest req = RequestConverter.buildTruncateTableRequest( - tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce()); + tableName, preserveSplits); master.truncateTable(null, req); return null; } @@ -1062,8 +1056,7 @@ public class HBaseAdmin implements Admin { @Override public EnableTableResponse call(int callTimeout) throws ServiceException { LOG.info("Started enable of " + tableName); - EnableTableRequest req = - RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); + EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName); return master.enableTable(null,req); } }); @@ -1250,8 +1243,7 @@ public class HBaseAdmin implements Admin { @Override public DisableTableResponse call(int callTimeout) throws ServiceException { LOG.info("Started disable of " + tableName); - DisableTableRequest req = - RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()); + DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName); return master.disableTable(null, req); } }); @@ -1530,8 +1522,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - AddColumnRequest req = RequestConverter.buildAddColumnRequest( - tableName, column, ng.getNonceGroup(), ng.newNonce()); + AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column); master.addColumn(null,req); return null; } @@ -1578,8 +1569,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest( - tableName, columnName, ng.getNonceGroup(), ng.newNonce()); + DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName); master.deleteColumn(null,req); return null; } @@ -1628,8 +1618,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest( - tableName, descriptor, ng.getNonceGroup(), ng.newNonce()); + ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor); master.modifyColumn(null,req); return null; } @@ -2464,8 +2453,7 @@ public class HBaseAdmin implements Admin { executeCallable(new MasterCallable(getConnection()) { @Override public Void call(int callTimeout) throws ServiceException { - ModifyTableRequest request = RequestConverter.buildModifyTableRequest( - tableName, htd, ng.getNonceGroup(), ng.newNonce()); + ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd); 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 e9e54ba1126..fdc40a5451b 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,15 +1061,10 @@ public final class RequestConverter { * @return an AddColumnRequest */ public static AddColumnRequest buildAddColumnRequest( - final TableName tableName, - final HColumnDescriptor column, - final long nonceGroup, - final long nonce) { + final TableName tableName, final HColumnDescriptor column) { AddColumnRequest.Builder builder = AddColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); builder.setColumnFamilies(column.convert()); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1081,15 +1076,10 @@ public final class RequestConverter { * @return a DeleteColumnRequest */ public static DeleteColumnRequest buildDeleteColumnRequest( - final TableName tableName, - final byte [] columnName, - final long nonceGroup, - final long nonce) { + final TableName tableName, final byte [] columnName) { DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setColumnName(ByteStringer.wrap(columnName)); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1101,15 +1091,10 @@ public final class RequestConverter { * @return an ModifyColumnRequest */ public static ModifyColumnRequest buildModifyColumnRequest( - final TableName tableName, - final HColumnDescriptor column, - final long nonceGroup, - final long nonce) { + final TableName tableName, final HColumnDescriptor column) { ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setColumnFamilies(column.convert()); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1191,14 +1176,9 @@ public final class RequestConverter { * @param tableName * @return a DeleteTableRequest */ - public static DeleteTableRequest buildDeleteTableRequest( - final TableName tableName, - final long nonceGroup, - final long nonce) { + public static DeleteTableRequest buildDeleteTableRequest(final TableName tableName) { DeleteTableRequest.Builder builder = DeleteTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1209,16 +1189,11 @@ public final class RequestConverter { * @param preserveSplits True if the splits should be preserved * @return a TruncateTableRequest */ - public static TruncateTableRequest buildTruncateTableRequest( - final TableName tableName, - final boolean preserveSplits, - final long nonceGroup, - final long nonce) { + public static TruncateTableRequest buildTruncateTableRequest(final TableName tableName, + boolean preserveSplits) { TruncateTableRequest.Builder builder = TruncateTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); builder.setPreserveSplits(preserveSplits); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1228,14 +1203,9 @@ public final class RequestConverter { * @param tableName * @return an EnableTableRequest */ - public static EnableTableRequest buildEnableTableRequest( - final TableName tableName, - final long nonceGroup, - final long nonce) { + public static EnableTableRequest buildEnableTableRequest(final TableName tableName) { EnableTableRequest.Builder builder = EnableTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName(tableName)); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1245,14 +1215,9 @@ public final class RequestConverter { * @param tableName * @return a DisableTableRequest */ - public static DisableTableRequest buildDisableTableRequest( - final TableName tableName, - final long nonceGroup, - final long nonce) { + public static DisableTableRequest buildDisableTableRequest(final TableName tableName) { DisableTableRequest.Builder builder = DisableTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1264,10 +1229,7 @@ public final class RequestConverter { * @return a CreateTableRequest */ public static CreateTableRequest buildCreateTableRequest( - final HTableDescriptor hTableDesc, - final byte [][] splitKeys, - final long nonceGroup, - final long nonce) { + final HTableDescriptor hTableDesc, final byte [][] splitKeys) { CreateTableRequest.Builder builder = CreateTableRequest.newBuilder(); builder.setTableSchema(hTableDesc.convert()); if (splitKeys != null) { @@ -1275,8 +1237,6 @@ public final class RequestConverter { builder.addSplitKeys(ByteStringer.wrap(splitKey)); } } - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1289,15 +1249,10 @@ public final class RequestConverter { * @return a ModifyTableRequest */ public static ModifyTableRequest buildModifyTableRequest( - final TableName tableName, - final HTableDescriptor hTableDesc, - final long nonceGroup, - final long nonce) { + final TableName tableName, final HTableDescriptor hTableDesc) { ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder(); builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setTableSchema(hTableDesc.convert()); - builder.setNonceGroup(nonceGroup); - builder.setNonce(nonce); return builder.build(); } @@ -1399,9 +1354,7 @@ 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 deleted file mode 100644 index 9c7c72ae1a5..00000000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/NonceKey.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * - * 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 a343c8957c9..601ebcd0f49 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,7 +28,6 @@ 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; @@ -36,7 +35,6 @@ 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; @@ -80,8 +78,6 @@ 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 @@ -266,10 +262,6 @@ 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. @@ -421,15 +413,6 @@ 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(). @@ -678,11 +661,6 @@ 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(); } @@ -734,11 +712,6 @@ 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 01e9a374813..49b658bf473 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,7 +39,6 @@ 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; @@ -49,7 +48,6 @@ 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; @@ -136,17 +134,14 @@ 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 nonceKeysToProcIdsMap) { + final Map completedMap) { // 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; } @@ -176,11 +171,6 @@ public class ProcedureExecutor { } store.delete(entry.getKey()); it.remove(); - - NonceKey nonceKey = result.getNonceKey(); - if (nonceKey != null) { - nonceKeysToProcIdsMap.remove(nonceKey); - } } } } @@ -234,13 +224,6 @@ 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. @@ -329,11 +312,6 @@ 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++; } @@ -365,7 +343,6 @@ public class ProcedureExecutor { assert !rollbackStack.containsKey(proc.getProcId()); procedures.remove(proc.getProcId()); completed.put(proc.getProcId(), newResultFromProcedure(proc)); - continue; } @@ -502,8 +479,7 @@ public class ProcedureExecutor { } // Add completed cleaner - waitingTimeout.add( - new CompletedProcedureCleaner(conf, store, completed, nonceKeysToProcIdsMap)); + waitingTimeout.add(new CompletedProcedureCleaner(conf, store, completed)); } public void stop() { @@ -534,7 +510,6 @@ public class ProcedureExecutor { completed.clear(); rollbackStack.clear(); procedures.clear(); - nonceKeysToProcIdsMap.clear(); waitingTimeout.clear(); runnables.clear(); lastProcId.set(-1); @@ -577,53 +552,13 @@ 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()); - 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) + // Initialize the Procedure ID + proc.setProcId(nextProcId()); // Commit the transaction store.insert(proc, null); @@ -633,14 +568,14 @@ public class ProcedureExecutor { // Create the rollback stack for the procedure RootProcedureState stack = new RootProcedureState(); - rollbackStack.put(currentProcId, stack); + rollbackStack.put(proc.getProcId(), stack); // Submit the new subprocedures - assert !procedures.containsKey(currentProcId); - procedures.put(currentProcId, proc); - sendProcedureAddedNotification(currentProcId); + assert !procedures.containsKey(proc.getProcId()); + procedures.put(proc.getProcId(), proc); + sendProcedureAddedNotification(proc.getProcId()); runnables.addBack(proc); - return currentProcId; + return proc.getProcId(); } public ProcedureResult getResult(final long procId) { @@ -1227,10 +1162,8 @@ public class ProcedureExecutor { private static ProcedureResult newResultFromProcedure(final Procedure proc) { if (proc.isFailed()) { - return new ProcedureResult( - proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getException()); + return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getException()); } - return new ProcedureResult( - proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getResult()); + return new ProcedureResult(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 ff5407f9c87..98c293be1c3 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,7 +20,6 @@ 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 @@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.NonceKey; @InterfaceAudience.Private @InterfaceStability.Evolving public class ProcedureResult { - private final NonceKey nonceKey; private final RemoteProcedureException exception; private final long lastUpdate; private final long startTime; @@ -39,37 +37,19 @@ public class ProcedureResult { private long clientAckTime = -1; - public ProcedureResult( - final NonceKey nonceKey, - final long startTime, - final long lastUpdate, + public ProcedureResult(final long startTime, final long lastUpdate, final RemoteProcedureException exception) { - this(nonceKey, exception, lastUpdate, startTime, null); - } - - 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.result = result; + this.exception = exception; + this.result = null; } - public NonceKey getNonceKey() { - return nonceKey; + public ProcedureResult(final long startTime, final long lastUpdate, final byte[] result) { + this.lastUpdate = lastUpdate; + this.startTime = startTime; + this.exception = null; + this.result = result; } public boolean isFailed() { 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 34774ed8c28..66eb3ea6a4a 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,7 +27,6 @@ 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; @@ -120,7 +119,7 @@ public class ProcedureTestingUtility { procStore.start(1); procExecutor.start(1, false); try { - return submitAndWait(procExecutor, proc, HConstants.NO_NONCE, HConstants.NO_NONCE); + return submitAndWait(procExecutor, proc); } finally { procStore.stop(false); procExecutor.stop(); @@ -128,13 +127,7 @@ public class ProcedureTestingUtility { } public static long submitAndWait(ProcedureExecutor procExecutor, Procedure 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); + long procId = procExecutor.submitProcedure(proc); 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 145f7f3564e..7735b63f435 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,6 +34,7 @@ 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; @@ -75,9 +76,6 @@ 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 @@ -287,41 +285,6 @@ 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 1a901280fcc..efa6a18a54e 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,26 +38,6 @@ 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} @@ -136,16 +116,6 @@ 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) { @@ -230,43 +200,9 @@ 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() { @@ -302,12 +238,6 @@ 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); } @@ -325,14 +255,6 @@ 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; @@ -366,16 +288,6 @@ 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; @@ -397,14 +309,6 @@ 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; @@ -528,10 +432,6 @@ public final class MasterProtos { columnFamiliesBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -576,14 +476,6 @@ 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; @@ -606,12 +498,6 @@ 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; } @@ -889,72 +775,6 @@ 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) } @@ -1330,26 +1150,6 @@ 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} @@ -1420,16 +1220,6 @@ 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) { @@ -1508,43 +1298,9 @@ 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() { @@ -1576,12 +1332,6 @@ 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); } @@ -1599,14 +1349,6 @@ 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; @@ -1640,16 +1382,6 @@ 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; @@ -1671,14 +1403,6 @@ 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; @@ -1797,10 +1521,6 @@ 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; } @@ -1841,14 +1561,6 @@ 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; @@ -1871,12 +1583,6 @@ 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; } @@ -2069,72 +1775,6 @@ 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) } @@ -2514,26 +2154,6 @@ 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} @@ -2612,16 +2232,6 @@ 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) { @@ -2706,43 +2316,9 @@ 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() { @@ -2778,12 +2354,6 @@ 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); } @@ -2801,14 +2371,6 @@ 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; @@ -2842,16 +2404,6 @@ 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; @@ -2873,14 +2425,6 @@ 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; @@ -3004,10 +2548,6 @@ public final class MasterProtos { columnFamiliesBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -3052,14 +2592,6 @@ 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; @@ -3082,12 +2614,6 @@ 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; } @@ -3365,72 +2891,6 @@ 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) } @@ -8925,26 +8385,6 @@ 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} @@ -9018,16 +8458,6 @@ 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) { @@ -9116,43 +8546,9 @@ 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() { @@ -9180,12 +8576,6 @@ 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); } @@ -9208,14 +8598,6 @@ 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; @@ -9246,16 +8628,6 @@ 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; @@ -9277,14 +8649,6 @@ 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; @@ -9403,10 +8767,6 @@ 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; } @@ -9448,14 +8808,6 @@ 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; @@ -9485,12 +8837,6 @@ public final class MasterProtos { } onChanged(); } - if (other.hasNonceGroup()) { - setNonceGroup(other.getNonceGroup()); - } - if (other.hasNonce()) { - setNonce(other.getNonce()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -9715,72 +9061,6 @@ 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) } @@ -10242,26 +9522,6 @@ 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} @@ -10327,16 +9587,6 @@ 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) { @@ -10399,42 +9649,8 @@ 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() { @@ -10459,12 +9675,6 @@ 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); } @@ -10478,14 +9688,6 @@ 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; @@ -10514,16 +9716,6 @@ 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; @@ -10541,14 +9733,6 @@ 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; @@ -10665,10 +9849,6 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -10705,14 +9885,6 @@ 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; @@ -10732,12 +9904,6 @@ 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; } @@ -10890,72 +10056,6 @@ 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) } @@ -11427,26 +10527,6 @@ 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} @@ -11517,16 +10597,6 @@ 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) { @@ -11605,43 +10675,9 @@ 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() { @@ -11669,12 +10705,6 @@ 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); } @@ -11692,14 +10722,6 @@ 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; @@ -11733,16 +10755,6 @@ 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; @@ -11764,14 +10776,6 @@ 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; @@ -11890,10 +10894,6 @@ public final class MasterProtos { bitField0_ = (bitField0_ & ~0x00000001); preserveSplits_ = false; bitField0_ = (bitField0_ & ~0x00000002); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -11934,14 +10934,6 @@ 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; @@ -11964,12 +10956,6 @@ 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; } @@ -12155,72 +11141,6 @@ 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) } @@ -12586,26 +11506,6 @@ 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} @@ -12671,16 +11571,6 @@ 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) { @@ -12743,42 +11633,8 @@ 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() { @@ -12803,12 +11659,6 @@ 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); } @@ -12822,14 +11672,6 @@ 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; @@ -12858,16 +11700,6 @@ 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; @@ -12885,14 +11717,6 @@ 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; @@ -13009,10 +11833,6 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -13049,14 +11869,6 @@ 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; @@ -13076,12 +11888,6 @@ 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; } @@ -13234,72 +12040,6 @@ 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) } @@ -13761,26 +12501,6 @@ 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} @@ -13846,16 +12566,6 @@ 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) { @@ -13918,42 +12628,8 @@ 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() { @@ -13978,12 +12654,6 @@ 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); } @@ -13997,14 +12667,6 @@ 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; @@ -14033,16 +12695,6 @@ 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; @@ -14060,14 +12712,6 @@ 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; @@ -14184,10 +12828,6 @@ public final class MasterProtos { tableNameBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -14224,14 +12864,6 @@ 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; @@ -14251,12 +12883,6 @@ 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; } @@ -14409,72 +13035,6 @@ 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) } @@ -14950,26 +13510,6 @@ 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} @@ -15048,16 +13588,6 @@ 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) { @@ -15142,43 +13672,9 @@ 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() { @@ -15214,12 +13710,6 @@ 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); } @@ -15237,14 +13727,6 @@ 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; @@ -15278,16 +13760,6 @@ 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; @@ -15309,14 +13781,6 @@ 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; @@ -15440,10 +13904,6 @@ public final class MasterProtos { tableSchemaBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); - nonceGroup_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - nonce_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -15488,14 +13948,6 @@ 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; @@ -15518,12 +13970,6 @@ 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; } @@ -15801,72 +14247,6 @@ 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) } @@ -53529,237 +51909,228 @@ 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\"\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" + "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" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -53771,7 +52142,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", "ColumnFamilies", }); internal_static_AddColumnResponse_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_AddColumnResponse_fieldAccessorTable = new @@ -53783,7 +52154,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", "ColumnName", }); internal_static_DeleteColumnResponse_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_DeleteColumnResponse_fieldAccessorTable = new @@ -53795,7 +52166,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", "ColumnFamilies", }); internal_static_ModifyColumnResponse_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_ModifyColumnResponse_fieldAccessorTable = new @@ -53867,7 +52238,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableSchema", "SplitKeys", }); internal_static_CreateTableResponse_descriptor = getDescriptor().getMessageTypes().get(17); internal_static_CreateTableResponse_fieldAccessorTable = new @@ -53879,7 +52250,7 @@ public final class MasterProtos { internal_static_DeleteTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DeleteTableRequest_descriptor, - new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", }); internal_static_DeleteTableResponse_descriptor = getDescriptor().getMessageTypes().get(19); internal_static_DeleteTableResponse_fieldAccessorTable = new @@ -53891,7 +52262,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", "PreserveSplits", }); internal_static_TruncateTableResponse_descriptor = getDescriptor().getMessageTypes().get(21); internal_static_TruncateTableResponse_fieldAccessorTable = new @@ -53903,7 +52274,7 @@ public final class MasterProtos { internal_static_EnableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableTableRequest_descriptor, - new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", }); internal_static_EnableTableResponse_descriptor = getDescriptor().getMessageTypes().get(23); internal_static_EnableTableResponse_fieldAccessorTable = new @@ -53915,7 +52286,7 @@ public final class MasterProtos { internal_static_DisableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DisableTableRequest_descriptor, - new java.lang.String[] { "TableName", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", }); internal_static_DisableTableResponse_descriptor = getDescriptor().getMessageTypes().get(25); internal_static_DisableTableResponse_fieldAccessorTable = new @@ -53927,7 +52298,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "TableName", "TableSchema", }); 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 d9d6ccb8dae..3c7dcdba324 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,34 +382,6 @@ 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} @@ -557,16 +529,6 @@ 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) { @@ -937,46 +899,6 @@ 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; @@ -990,8 +912,6 @@ 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() { @@ -1061,12 +981,6 @@ 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); } @@ -1129,14 +1043,6 @@ 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; @@ -1217,16 +1123,6 @@ 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; @@ -1288,14 +1184,6 @@ 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; @@ -1439,10 +1327,6 @@ 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; } @@ -1524,14 +1408,6 @@ 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; @@ -1595,12 +1471,6 @@ 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; } @@ -2404,88 +2274,6 @@ 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) } @@ -7336,34 +7124,33 @@ public final class ProcedureProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\017Procedure.proto\032\023ErrorHandling.proto\"\271" + + "\n\017Procedure.proto\032\023ErrorHandling.proto\"\217" + "\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\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" + "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" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -7375,7 +7162,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", "NonceGroup", "Nonce", }); + new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", }); 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 469c0a40dd1..0b2e875a3e5 100644 --- a/hbase-protocol/src/main/protobuf/Master.proto +++ b/hbase-protocol/src/main/protobuf/Master.proto @@ -36,8 +36,6 @@ 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 { @@ -46,8 +44,6 @@ 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 { @@ -56,8 +52,6 @@ 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 { @@ -112,8 +106,6 @@ 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 { @@ -122,8 +114,6 @@ message CreateTableResponse { message DeleteTableRequest { required TableName table_name = 1; - optional uint64 nonce_group = 2 [default = 0]; - optional uint64 nonce = 3 [default = 0]; } message DeleteTableResponse { @@ -133,8 +123,6 @@ 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 { @@ -142,8 +130,6 @@ message TruncateTableResponse { message EnableTableRequest { required TableName table_name = 1; - optional uint64 nonce_group = 2 [default = 0]; - optional uint64 nonce = 3 [default = 0]; } message EnableTableResponse { @@ -152,8 +138,6 @@ message EnableTableResponse { message DisableTableRequest { required TableName table_name = 1; - optional uint64 nonce_group = 2 [default = 0]; - optional uint64 nonce = 3 [default = 0]; } message DisableTableResponse { @@ -163,8 +147,6 @@ 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 46ff9670d97..232c2903d4d 100644 --- a/hbase-protocol/src/main/protobuf/Procedure.proto +++ b/hbase-protocol/src/main/protobuf/Procedure.proto @@ -54,10 +54,6 @@ 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 bb2470c868b..3e5d9094df2 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,11 +1435,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long createTable( - final HTableDescriptor hTableDescriptor, - final byte [][] splitKeys, - final long nonceGroup, - final long nonce) throws IOException { + public long createTable(HTableDescriptor hTableDescriptor, + byte [][] splitKeys) throws IOException { if (isStopped()) { throw new MasterNotRunningException(); } @@ -1460,10 +1457,8 @@ 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), - nonceGroup, - nonce); + new CreateTableProcedure(procedureExecutor.getEnvironment(), + hTableDescriptor, newRegions, latch)); latch.await(); if (cpHost != null) { @@ -1701,10 +1696,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long deleteTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException { + public long deleteTable(final TableName tableName) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preDeleteTable(tableName); @@ -1714,9 +1706,7 @@ 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), - nonceGroup, - nonce); + new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch)); latch.await(); if (cpHost != null) { @@ -1727,11 +1717,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void truncateTable( - final TableName tableName, - final boolean preserveSplits, - final long nonceGroup, - final long nonce) throws IOException { + public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preTruncateTable(tableName); @@ -1739,9 +1725,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { LOG.info(getClientIdAuditPrefix() + " truncate " + tableName); long procId = this.procedureExecutor.submitProcedure( - new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits), - nonceGroup, - nonce); + new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits)); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1750,11 +1734,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void addColumn( - final TableName tableName, - final HColumnDescriptor columnDescriptor, - final long nonceGroup, - final long nonce) + public void addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor) throws IOException { checkInitialized(); checkCompression(columnDescriptor); @@ -1765,10 +1745,9 @@ 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), - nonceGroup, - nonce); + long procId = + this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor + .getEnvironment(), tableName, columnDescriptor)); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { cpHost.postAddColumn(tableName, columnDescriptor); @@ -1776,11 +1755,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void modifyColumn( - final TableName tableName, - final HColumnDescriptor descriptor, - final long nonceGroup, - final long nonce) + public void modifyColumn(TableName tableName, HColumnDescriptor descriptor) throws IOException { checkInitialized(); checkCompression(descriptor); @@ -1793,10 +1768,9 @@ 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), - nonceGroup, - nonce); + long procId = + this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor + .getEnvironment(), tableName, descriptor)); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1805,11 +1779,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void deleteColumn( - final TableName tableName, - final byte[] columnName, - final long nonceGroup, - final long nonce) + public void deleteColumn(final TableName tableName, final byte[] columnName) throws IOException { checkInitialized(); if (cpHost != null) { @@ -1820,10 +1790,9 @@ 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), - nonceGroup, - nonce); + long procId = + this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor + .getEnvironment(), tableName, columnName)); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); if (cpHost != null) { @@ -1832,10 +1801,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long enableTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException { + public long enableTable(final TableName tableName) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preEnableTable(tableName); @@ -1844,10 +1810,9 @@ 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), - nonceGroup, - nonce); + long procId = + this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor + .getEnvironment(), tableName, false, prepareLatch)); // 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). // @@ -1862,10 +1827,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public long disableTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException { + public long disableTable(final TableName tableName) throws IOException { checkInitialized(); if (cpHost != null) { cpHost.preDisableTable(tableName); @@ -1875,10 +1837,9 @@ 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), - nonceGroup, - nonce); + long procId = + this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor + .getEnvironment(), tableName, false, prepareLatch)); // 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). // @@ -1929,11 +1890,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void modifyTable( - final TableName tableName, - final HTableDescriptor descriptor, - final long nonceGroup, - final long nonce) + public void modifyTable(final TableName tableName, final HTableDescriptor descriptor) throws IOException { checkInitialized(); sanityCheckTableDescriptor(descriptor); @@ -1945,9 +1902,7 @@ 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), - nonceGroup, - nonce); + new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor)); ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId); @@ -2102,7 +2057,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 fb4509097f6..27fab4f2c04 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,11 +347,8 @@ 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()), - req.getNonceGroup(), - req.getNonce()); + master.addColumn(ProtobufUtil.toTableName(req.getTableName()), + HColumnDescriptor.convert(req.getColumnFamilies())); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -418,8 +415,7 @@ public class MasterRpcServices extends RSRpcServices HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema()); byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req); try { - long procId = - master.createTable(hTableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce()); + long procId = master.createTable(hTableDescriptor, splitKeys); return CreateTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -430,11 +426,8 @@ public class MasterRpcServices extends RSRpcServices public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req) throws ServiceException { try { - master.deleteColumn( - ProtobufUtil.toTableName(req.getTableName()), - req.getColumnName().toByteArray(), - req.getNonceGroup(), - req.getNonce()); + master.deleteColumn(ProtobufUtil.toTableName(req.getTableName()), + req.getColumnName().toByteArray()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -478,8 +471,7 @@ public class MasterRpcServices extends RSRpcServices public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request) throws ServiceException { try { - long procId = master.deleteTable(ProtobufUtil.toTableName( - request.getTableName()), request.getNonceGroup(), request.getNonce()); + long procId = master.deleteTable(ProtobufUtil.toTableName(request.getTableName())); return DeleteTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -490,11 +482,8 @@ public class MasterRpcServices extends RSRpcServices public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request) throws ServiceException { try { - master.truncateTable( - ProtobufUtil.toTableName(request.getTableName()), - request.getPreserveSplits(), - request.getNonceGroup(), - request.getNonce()); + master.truncateTable(ProtobufUtil.toTableName(request.getTableName()), + request.getPreserveSplits()); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -505,10 +494,7 @@ public class MasterRpcServices extends RSRpcServices public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request) throws ServiceException { try { - long procId = master.disableTable( - ProtobufUtil.toTableName(request.getTableName()), - request.getNonceGroup(), - request.getNonce()); + long procId = master.disableTable(ProtobufUtil.toTableName(request.getTableName())); return DisableTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -594,10 +580,7 @@ public class MasterRpcServices extends RSRpcServices public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request) throws ServiceException { try { - long procId = master.enableTable( - ProtobufUtil.toTableName(request.getTableName()), - request.getNonceGroup(), - request.getNonce()); + long procId = master.enableTable(ProtobufUtil.toTableName(request.getTableName())); return EnableTableResponse.newBuilder().setProcId(procId).build(); } catch (IOException ioe) { throw new ServiceException(ioe); @@ -1067,11 +1050,8 @@ 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()), - req.getNonceGroup(), - req.getNonce()); + master.modifyColumn(ProtobufUtil.toTableName(req.getTableName()), + HColumnDescriptor.convert(req.getColumnFamilies())); } catch (IOException ioe) { throw new ServiceException(ioe); } @@ -1094,11 +1074,8 @@ 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()), - req.getNonceGroup(), - req.getNonce()); + master.modifyTable(ProtobufUtil.toTableName(req.getTableName()), + HTableDescriptor.convert(req.getTableSchema())); } 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 c22f0ed3dfc..dd64bc84cd7 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,125 +98,75 @@ 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( - final HTableDescriptor desc, - final byte[][] splitKeys, - final long nonceGroup, - final long nonce) throws IOException; + long createTable(HTableDescriptor desc, byte[][] splitKeys) + throws IOException; /** * Delete a table * @param tableName The table name - * @param nonceGroup - * @param nonce * @throws IOException */ - long deleteTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException; + long deleteTable(final TableName tableName) 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, - final boolean preserveSplits, - final long nonceGroup, - final long nonce) throws IOException; + public void truncateTable(final TableName tableName, boolean preserveSplits) 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, - final long nonceGroup, - final long nonce) + void modifyTable(final TableName tableName, final HTableDescriptor descriptor) throws IOException; /** * Enable an existing table * @param tableName The table name - * @param nonceGroup - * @param nonce * @throws IOException */ - long enableTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException; + long enableTable(final TableName tableName) throws IOException; /** * Disable an existing table * @param tableName The table name - * @param nonceGroup - * @param nonce * @throws IOException */ - long disableTable( - final TableName tableName, - final long nonceGroup, - final long nonce) throws IOException; + long disableTable(final TableName tableName) 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, - final long nonceGroup, - final long nonce) + void addColumn(final TableName tableName, final HColumnDescriptor column) 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( - final TableName tableName, - final HColumnDescriptor descriptor, - final long nonceGroup, - final long nonce) + void modifyColumn(TableName tableName, HColumnDescriptor descriptor) 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, - final long nonceGroup, - final long nonce) + void deleteColumn(final TableName tableName, final byte[] columnName) 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 b2b656b1d81..0d974b1edd8 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,7 +31,6 @@ 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; @@ -103,6 +102,38 @@ 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 4ccf4dc6aca..131ff14ddcc 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,9 +136,7 @@ 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, - HConstants.NO_NONCE, - HConstants.NO_NONCE); + null); } /** 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 7d3b82e71ae..442f7eabacf 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, HConstants.NO_NONCE, HConstants.NO_NONCE); + master.createTable(labelsTable, null); } } 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 19774d74fa8..6672eb247e5 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,7 +45,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Matchers; @@ -67,10 +66,6 @@ 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 343cd4cf61b..29766b3315f 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(), HConstants.NO_NONCE, HConstants.NO_NONCE); + master.disableTable(hri.getTable()); // 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 fbdf6a2e457..053dc994fe2 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,11 +224,8 @@ public class TestCatalogJanitor { } @Override - public long createTable( - final HTableDescriptor desc, - final byte[][] splitKeys, - final long nonceGroup, - final long nonce) throws IOException { + public long createTable(HTableDescriptor desc, byte[][] splitKeys) + throws IOException { // no-op return -1; } @@ -257,7 +254,7 @@ public class TestCatalogJanitor { public MasterCoprocessorHost getMasterCoprocessorHost() { return null; } - + @Override public MasterQuotaManager getMasterQuotaManager() { return null; @@ -412,64 +409,39 @@ public class TestCatalogJanitor { } @Override - 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 { + public long deleteTable(TableName tableName) throws IOException { return -1; } @Override - public long disableTable( - TableName tableName, - final long nonceGroup, - final long nonce) throws IOException { + 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 { return -1; } @Override - public void addColumn( - final TableName tableName, - final HColumnDescriptor columnDescriptor, - final long nonceGroup, - final long nonce) throws IOException { } + public long disableTable(TableName tableName) throws IOException { + return -1; + } @Override - public void modifyColumn( - final TableName tableName, - final HColumnDescriptor descriptor, - final long nonceGroup, - final long nonce) throws IOException { } + public void addColumn(TableName tableName, HColumnDescriptor column) + throws IOException { } @Override - public void deleteColumn( - final TableName tableName, - final byte[] columnName, - final long nonceGroup, - final long nonce) throws IOException { } + public void modifyColumn(TableName tableName, HColumnDescriptor descriptor) + throws IOException { } + + @Override + public void deleteColumn(TableName tableName, byte[] columnName) + 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 77ca96e445f..00f82f4ce8a 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,7 +38,6 @@ 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; @@ -426,14 +425,6 @@ 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 e09c8386496..05eb6027083 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,7 +25,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.TableName; @@ -47,9 +46,6 @@ 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); } @@ -72,9 +68,6 @@ 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 @@ -98,10 +91,9 @@ 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), - nonceGroup, - nonce); + long procId1 = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor1)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -111,10 +103,9 @@ public class TestAddColumnFamilyProcedure { // Test 2: Add a column family offline UTIL.getHBaseAdmin().disableTable(tableName); - long procId2 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2), - nonceGroup + 1, - nonce + 1); + long procId2 = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor2)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -133,10 +124,9 @@ public class TestAddColumnFamilyProcedure { MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1"); // add the column family - long procId1 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId1 = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -144,10 +134,9 @@ public class TestAddColumnFamilyProcedure { tableName, cf2); // add the column family that exists - long procId2 = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup + 1, - nonce + 1); + long procId2 = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); @@ -159,10 +148,9 @@ 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), - nonceGroup + 2, - nonce + 2); + long procId3 = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId3); @@ -173,37 +161,6 @@ 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"); @@ -218,10 +175,9 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); // Restart the executor and execute the step twice int numberOfSteps = AddColumnFamilyState.values().length; @@ -245,10 +201,9 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); // Restart the executor and execute the step twice int numberOfSteps = AddColumnFamilyState.values().length; @@ -272,10 +227,9 @@ public class TestAddColumnFamilyProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the AddColumnFamily procedure && kill the executor - long procId = procExec.submitProcedure( - new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, + columnDescriptor)); 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 73843e0e79d..21129f27b27 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,7 +24,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableExistsException; @@ -35,6 +34,7 @@ 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,7 +42,10 @@ 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 { @@ -50,9 +53,6 @@ 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,9 +75,6 @@ public class TestCreateTableProcedure { @Before public void setup() throws Exception { resetProcExecutorTestingKillFlag(); - nonceGroup = - MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster()); - nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster()); } @After @@ -127,14 +124,12 @@ public class TestCreateTableProcedure { // create the table long procId1 = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); // create another with the same name ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch(); long procId2 = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2), - nonceGroup + 1, - nonce + 1); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2)); ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1)); @@ -143,29 +138,6 @@ 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"); @@ -179,7 +151,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), nonceGroup, nonce); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); // Restart the executor and execute the step twice // NOTE: the 6 (number of CreateTableState steps) is hardcoded, @@ -207,7 +179,7 @@ public class TestCreateTableProcedure { htd.setRegionReplication(3); HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys); long procId = procExec.submitProcedure( - new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce); + new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); // 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. @@ -237,7 +209,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), nonceGroup, nonce); + new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions)); // 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 e7efc2b5b29..5aec002a0f8 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,7 +24,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; @@ -47,9 +46,6 @@ 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); } @@ -72,9 +68,6 @@ 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 @@ -96,10 +89,9 @@ 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()), - nonceGroup, - nonce); + long procId1 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf1.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -109,10 +101,9 @@ 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()), - nonceGroup, - nonce); + long procId2 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf2.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -128,10 +119,9 @@ 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()), - nonceGroup, - nonce); + long procId1 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf2.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); // First delete should succeed @@ -141,10 +131,9 @@ public class TestDeleteColumnFamilyProcedure { tableName, cf2); // delete the column family that does not exist - long procId2 = procExec.submitProcedure( - new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()), - nonceGroup + 1, - nonce + 1); + long procId2 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf2.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); @@ -157,10 +146,9 @@ 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()), - nonceGroup + 2, - nonce + 2); + long procId3 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf2.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId3); // Expect fail with InvalidFamilyOperationException @@ -170,37 +158,6 @@ 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"); @@ -211,10 +168,9 @@ 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()), - nonceGroup, - nonce); + long procId1 = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf3.getBytes())); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -238,10 +194,9 @@ 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()), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf4.getBytes())); // Restart the executor and execute the step twice int numberOfSteps = DeleteColumnFamilyState.values().length; @@ -265,10 +220,9 @@ 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()), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(), + tableName, cf5.getBytes())); // Restart the executor and execute the step twice int numberOfSteps = DeleteColumnFamilyState.values().length; @@ -294,9 +248,7 @@ public class TestDeleteColumnFamilyProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()), - nonceGroup, - nonce); + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes())); // 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, @@ -327,9 +279,7 @@ public class TestDeleteColumnFamilyProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()), - nonceGroup, - nonce); + new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes())); // 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 a215c21148c..dc1d90a3c1c 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,7 +22,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -34,6 +33,7 @@ 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,7 +41,10 @@ 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 { @@ -49,9 +52,6 @@ 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,10 +76,6 @@ 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 @@ -126,10 +122,10 @@ public class TestDeleteTableProcedure { // delete the table (that exists) long procId1 = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); + new DeleteTableProcedure(procExec.getEnvironment(), tableName)); // delete the table (that will no longer exist) long procId2 = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup + 1, nonce + 1); + new DeleteTableProcedure(procExec.getEnvironment(), tableName)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -147,36 +143,6 @@ 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"); @@ -223,7 +189,7 @@ public class TestDeleteTableProcedure { // Start the Delete procedure && kill the executor long procId = procExec.submitProcedure( - new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce); + new DeleteTableProcedure(procExec.getEnvironment(), tableName)); // 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 7f49e11bbcb..d588a3b76a2 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,7 +24,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; @@ -34,6 +33,7 @@ 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,9 +48,6 @@ 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); } @@ -73,9 +70,6 @@ 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 @@ -96,7 +90,7 @@ public class TestDisableTableProcedure { // Disable the table long procId = procExec.submitProcedure( - new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + new DisableTableProcedure(procExec.getEnvironment(), tableName, false)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId); @@ -113,7 +107,7 @@ public class TestDisableTableProcedure { // Disable the table long procId1 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + procExec.getEnvironment(), tableName, false)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -122,7 +116,7 @@ public class TestDisableTableProcedure { // Disable the table again - expect failure long procId2 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false), nonceGroup + 1, nonce + 1); + procExec.getEnvironment(), tableName, false)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureResult result = procExec.getResult(procId2); @@ -135,7 +129,7 @@ public class TestDisableTableProcedure { final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch(); long procId3 = procExec.submitProcedure(new DisableTableProcedure( - procExec.getEnvironment(), tableName, false, prepareLatch), nonceGroup + 2, nonce + 2); + procExec.getEnvironment(), tableName, false, prepareLatch)); prepareLatch.await(); Assert.fail("Disable should throw exception through latch."); } catch (TableNotEnabledException tnee) { @@ -153,29 +147,6 @@ 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"); @@ -189,8 +160,9 @@ public class TestDisableTableProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Start the Disable procedure && kill the executor - long procId = procExec.submitProcedure( - new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new DisableTableProcedure(procExec.getEnvironment(), tableName, + false)); // 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 7f8687abd45..8964adcf84a 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,7 +24,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; @@ -34,6 +33,7 @@ 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,9 +48,6 @@ 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); } @@ -73,9 +70,6 @@ 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 @@ -97,7 +91,7 @@ public class TestEnableTableProcedure { // Enable the table long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId); @@ -105,29 +99,6 @@ 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"); @@ -137,7 +108,7 @@ public class TestEnableTableProcedure { // Enable the table - expect failure long procId1 = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureResult result = procExec.getResult(procId1); @@ -147,9 +118,7 @@ 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), - nonceGroup + 1, - nonce + 1); + new EnableTableProcedure(procExec.getEnvironment(), tableName, true)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -157,9 +126,7 @@ 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), - nonceGroup + 2, - nonce + 2); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch)); prepareLatch.await(); Assert.fail("Enable should throw exception through latch."); } @@ -179,7 +146,7 @@ public class TestEnableTableProcedure { // Start the Enable procedure && kill the executor long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); // Restart the executor and execute the step twice int numberOfSteps = EnableTableState.values().length; @@ -207,7 +174,7 @@ public class TestEnableTableProcedure { // Start the Enable procedure && kill the executor long procId = procExec.submitProcedure( - new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce); + new EnableTableProcedure(procExec.getEnvironment(), tableName, false)); 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 c7f05f1f8d7..ea8882f1bf4 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,7 +25,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.TableName; @@ -47,9 +46,6 @@ 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); } @@ -72,9 +68,6 @@ 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 @@ -100,10 +93,8 @@ public class TestModifyColumnFamilyProcedure { // Test 1: modify the column family online columnDescriptor.setBlocksize(newBlockSize); - long procId1 = procExec.submitProcedure( - new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure( + procExec.getEnvironment(), tableName, columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); ProcedureTestingUtility.assertProcNotFailed(procExec, procId1); @@ -113,10 +104,9 @@ 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), - nonceGroup + 1, - nonce + 1); + long procId2 = + procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(), + tableName, columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId2); ProcedureTestingUtility.assertProcNotFailed(procExec, procId2); @@ -138,10 +128,8 @@ public class TestModifyColumnFamilyProcedure { // Modify the column family that does not exist columnDescriptor.setBlocksize(newBlockSize); - long procId1 = procExec.submitProcedure( - new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure( + procExec.getEnvironment(), tableName, columnDescriptor)); // Wait the completion ProcedureTestingUtility.waitProcedure(procExec, procId1); @@ -169,10 +157,8 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = procExec.submitProcedure( - new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure( + procExec.getEnvironment(), tableName, columnDescriptor)); // Restart the executor and execute the step twice int numberOfSteps = ModifyColumnFamilyState.values().length; @@ -203,10 +189,9 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = procExec.submitProcedure( - new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = + procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(), + tableName, columnDescriptor)); // Restart the executor and execute the step twice int numberOfSteps = ModifyColumnFamilyState.values().length; @@ -234,10 +219,8 @@ public class TestModifyColumnFamilyProcedure { // Start the Modify procedure && kill the executor columnDescriptor.setBlocksize(newBlockSize); - long procId = procExec.submitProcedure( - new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor), - nonceGroup, - nonce); + long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure( + procExec.getEnvironment(), tableName, columnDescriptor)); // 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 59229d47de3..ebc238f8812 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,7 +27,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -48,9 +47,6 @@ 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); } @@ -73,9 +69,6 @@ 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 @@ -230,8 +223,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = procExec.submitProcedure( - new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); // Restart the executor and execute the step twice int numberOfSteps = ModifyTableState.values().length; @@ -272,8 +265,8 @@ public class TestModifyTableProcedure { htd.removeFamily(cf3.getBytes()); // Start the Modify procedure && kill the executor - long procId = procExec.submitProcedure( - new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); // Restart the executor and execute the step twice int numberOfSteps = ModifyTableState.values().length; @@ -310,8 +303,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), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); // Restart the executor and rollback the step twice int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc @@ -347,8 +340,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = procExec.submitProcedure( - new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); // Restart the executor and rollback the step twice int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc @@ -385,8 +378,8 @@ public class TestModifyTableProcedure { htd.setRegionReplication(3); // Start the Modify procedure && kill the executor - long procId = procExec.submitProcedure( - new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce); + long procId = + procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd)); // 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 2500ccae260..1646ba0b944 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,7 +22,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; @@ -34,6 +33,7 @@ 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,7 +42,9 @@ 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 { @@ -50,9 +52,6 @@ public class TestTruncateTableProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private static long nonceGroup = HConstants.NO_NONCE; - private static long nonce = HConstants.NO_NONCE; - private static void setupConf(Configuration conf) { conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); } @@ -77,10 +76,6 @@ 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 @@ -214,9 +209,7 @@ public class TestTruncateTableProcedure { // Start the Truncate procedure && kill the executor long procId = procExec.submitProcedure( - new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits), - nonceGroup, - nonce); + new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits)); // Restart the executor and execute the step twice // NOTE: the 7 (number of TruncateTableState steps) is hardcoded,