HBASE-13415 Procedure v2 - Use nonces for double submits from client (Stephen Yuan Jiang)

Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
Matteo Bertozzi 2015-07-09 07:44:36 -07:00 committed by Sean Busbey
parent 60d6435ad8
commit 951ec7a0b7
31 changed files with 3042 additions and 517 deletions

View File

@ -205,6 +205,8 @@ public class HBaseAdmin implements Admin {
private RpcRetryingCallerFactory rpcCallerFactory;
private NonceGenerator ng;
/**
* Constructor.
* See {@link #HBaseAdmin(Connection connection)}
@ -259,6 +261,8 @@ public class HBaseAdmin implements Admin {
"hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
this.ng = this.connection.getNonceGenerator();
}
@Override
@ -618,7 +622,8 @@ public class HBaseAdmin implements Admin {
new MasterCallable<CreateTableResponse>(getConnection()) {
@Override
public CreateTableResponse call(int callTimeout) throws ServiceException {
CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys);
CreateTableRequest request = RequestConverter.buildCreateTableRequest(
desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
return master.createTable(null, request);
}
});
@ -706,7 +711,8 @@ public class HBaseAdmin implements Admin {
new MasterCallable<DeleteTableResponse>(getConnection()) {
@Override
public DeleteTableResponse call(int callTimeout) throws ServiceException {
DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName);
DeleteTableRequest req =
RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
return master.deleteTable(null,req);
}
});
@ -829,9 +835,9 @@ public class HBaseAdmin implements Admin {
executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
@Override
public TruncateTableResponse call(int callTimeout) throws ServiceException {
LOG.info("Started enable of " + tableName);
TruncateTableRequest req =
RequestConverter.buildTruncateTableRequest(tableName, preserveSplits);
LOG.info("Started truncating " + tableName);
TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
return master.truncateTable(null, req);
}
});
@ -992,7 +998,8 @@ public class HBaseAdmin implements Admin {
@Override
public EnableTableResponse call(int callTimeout) throws ServiceException {
LOG.info("Started enable of " + tableName);
EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
EnableTableRequest req =
RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
return master.enableTable(null,req);
}
});
@ -1129,7 +1136,8 @@ public class HBaseAdmin implements Admin {
@Override
public DisableTableResponse call(int callTimeout) throws ServiceException {
LOG.info("Started disable of " + tableName);
DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
DisableTableRequest req =
RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
return master.disableTable(null, req);
}
});
@ -1411,7 +1419,8 @@ public class HBaseAdmin implements Admin {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, columnFamily);
AddColumnRequest req = RequestConverter.buildAddColumnRequest(
tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
master.addColumn(null, req);
return null;
}
@ -1479,8 +1488,8 @@ public class HBaseAdmin implements Admin {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
DeleteColumnRequest req =
RequestConverter.buildDeleteColumnRequest(tableName, columnFamily);
DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(
tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
master.deleteColumn(null, req);
return null;
}
@ -1548,8 +1557,8 @@ public class HBaseAdmin implements Admin {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
ModifyColumnRequest req =
RequestConverter.buildModifyColumnRequest(tableName, columnFamily);
ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(
tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
master.modifyColumn(null,req);
return null;
}
@ -2398,7 +2407,8 @@ public class HBaseAdmin implements Admin {
new MasterCallable<ModifyTableResponse>(getConnection()) {
@Override
public ModifyTableResponse call(int callTimeout) throws ServiceException {
ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd);
ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
tableName, htd, ng.getNonceGroup(), ng.newNonce());
return master.modifyTable(null, request);
}
});

View File

@ -1041,10 +1041,15 @@ public final class RequestConverter {
* @return an AddColumnRequest
*/
public static AddColumnRequest buildAddColumnRequest(
final TableName tableName, final HColumnDescriptor column) {
final TableName tableName,
final HColumnDescriptor column,
final long nonceGroup,
final long nonce) {
AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setColumnFamilies(column.convert());
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1056,10 +1061,15 @@ public final class RequestConverter {
* @return a DeleteColumnRequest
*/
public static DeleteColumnRequest buildDeleteColumnRequest(
final TableName tableName, final byte [] columnName) {
final TableName tableName,
final byte [] columnName,
final long nonceGroup,
final long nonce) {
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnName(ByteStringer.wrap(columnName));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1071,10 +1081,15 @@ public final class RequestConverter {
* @return an ModifyColumnRequest
*/
public static ModifyColumnRequest buildModifyColumnRequest(
final TableName tableName, final HColumnDescriptor column) {
final TableName tableName,
final HColumnDescriptor column,
final long nonceGroup,
final long nonce) {
ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnFamilies(column.convert());
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1156,9 +1171,14 @@ public final class RequestConverter {
* @param tableName
* @return a DeleteTableRequest
*/
public static DeleteTableRequest buildDeleteTableRequest(final TableName tableName) {
public static DeleteTableRequest buildDeleteTableRequest(
final TableName tableName,
final long nonceGroup,
final long nonce) {
DeleteTableRequest.Builder builder = DeleteTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1169,11 +1189,16 @@ public final class RequestConverter {
* @param preserveSplits True if the splits should be preserved
* @return a TruncateTableRequest
*/
public static TruncateTableRequest buildTruncateTableRequest(final TableName tableName,
boolean preserveSplits) {
public static TruncateTableRequest buildTruncateTableRequest(
final TableName tableName,
final boolean preserveSplits,
final long nonceGroup,
final long nonce) {
TruncateTableRequest.Builder builder = TruncateTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setPreserveSplits(preserveSplits);
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1183,9 +1208,14 @@ public final class RequestConverter {
* @param tableName
* @return an EnableTableRequest
*/
public static EnableTableRequest buildEnableTableRequest(final TableName tableName) {
public static EnableTableRequest buildEnableTableRequest(
final TableName tableName,
final long nonceGroup,
final long nonce) {
EnableTableRequest.Builder builder = EnableTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1195,9 +1225,14 @@ public final class RequestConverter {
* @param tableName
* @return a DisableTableRequest
*/
public static DisableTableRequest buildDisableTableRequest(final TableName tableName) {
public static DisableTableRequest buildDisableTableRequest(
final TableName tableName,
final long nonceGroup,
final long nonce) {
DisableTableRequest.Builder builder = DisableTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1209,7 +1244,10 @@ public final class RequestConverter {
* @return a CreateTableRequest
*/
public static CreateTableRequest buildCreateTableRequest(
final HTableDescriptor hTableDesc, final byte [][] splitKeys) {
final HTableDescriptor hTableDesc,
final byte [][] splitKeys,
final long nonceGroup,
final long nonce) {
CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
builder.setTableSchema(hTableDesc.convert());
if (splitKeys != null) {
@ -1217,6 +1255,8 @@ public final class RequestConverter {
builder.addSplitKeys(ByteStringer.wrap(splitKey));
}
}
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1229,10 +1269,15 @@ public final class RequestConverter {
* @return a ModifyTableRequest
*/
public static ModifyTableRequest buildModifyTableRequest(
final TableName tableName, final HTableDescriptor hTableDesc) {
final TableName tableName,
final HTableDescriptor hTableDesc,
final long nonceGroup,
final long nonce) {
ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setTableSchema(hTableDesc.convert());
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
@ -1347,7 +1392,9 @@ public final class RequestConverter {
* @param synchronous
* @return a SetBalancerRunningRequest
*/
public static SetBalancerRunningRequest buildSetBalancerRunningRequest(boolean on, boolean synchronous) {
public static SetBalancerRunningRequest buildSetBalancerRunningRequest(
boolean on,
boolean synchronous) {
return SetBalancerRunningRequest.newBuilder().setOn(on).setSynchronous(synchronous).build();
}

View File

@ -0,0 +1,65 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* This implementation is not smart and just treats nonce group and nonce as random bits.
*/
// TODO: we could use pure byte arrays, but then we wouldn't be able to use hash map.
@InterfaceAudience.Private
public class NonceKey {
private long group;
private long nonce;
public NonceKey(long group, long nonce) {
assert nonce != HConstants.NO_NONCE;
this.group = group;
this.nonce = nonce;
}
@Override
public boolean equals(Object obj) {
if (obj == null || !(obj instanceof NonceKey)) {
return false;
}
NonceKey nk = ((NonceKey)obj);
return this.nonce == nk.nonce && this.group == nk.group;
}
@Override
public int hashCode() {
return (int)((group >> 32) ^ group ^ (nonce >> 32) ^ nonce);
}
@Override
public String toString() {
return "[" + group + ":" + nonce + "]";
}
public long getNonceGroup() {
return group;
}
public long getNonce() {
return nonce;
}
}

View File

@ -28,6 +28,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.NonceKey;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -78,6 +80,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
private RemoteProcedureException exception = null;
private byte[] result = null;
private NonceKey nonceKey = null;
/**
* The main code of the procedure. It must be idempotent since execute()
* may be called multiple time in case of machine failure in the middle
@ -262,6 +266,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
return parentProcId;
}
public NonceKey getNonceKey() {
return nonceKey;
}
/**
* @return true if the procedure has failed.
* true may mean failed but not yet rolledback or failed and rolledback.
@ -413,6 +421,15 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
this.parentProcId = parentProcId;
}
/**
* Called by the ProcedureExecutor to set the value to the newly created procedure.
*/
@VisibleForTesting
@InterfaceAudience.Private
protected void setNonceKey(final NonceKey nonceKey) {
this.nonceKey = nonceKey;
}
/**
* Internal method called by the ProcedureExecutor that starts the
* user-level code execute().
@ -661,6 +678,11 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
builder.setStateData(stateStream.toByteString());
}
if (proc.getNonceKey() != null) {
builder.setNonceGroup(proc.getNonceKey().getNonceGroup());
builder.setNonce(proc.getNonceKey().getNonce());
}
return builder.build();
}
@ -712,6 +734,11 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
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());

View File

@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@ -48,6 +49,7 @@ import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue;
import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.base.Preconditions;
@ -134,14 +136,17 @@ public class ProcedureExecutor<TEnvironment> {
private static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min
private final Map<Long, ProcedureResult> completed;
private final Map<NonceKey, Long> nonceKeysToProcIdsMap;
private final ProcedureStore store;
private final Configuration conf;
public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store,
final Map<Long, ProcedureResult> completedMap) {
final Map<Long, ProcedureResult> completedMap,
final Map<NonceKey, Long> nonceKeysToProcIdsMap) {
// set the timeout interval that triggers the periodic-procedure
setTimeout(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL));
this.completed = completedMap;
this.nonceKeysToProcIdsMap = nonceKeysToProcIdsMap;
this.store = store;
this.conf = conf;
}
@ -171,6 +176,11 @@ public class ProcedureExecutor<TEnvironment> {
}
store.delete(entry.getKey());
it.remove();
NonceKey nonceKey = result.getNonceKey();
if (nonceKey != null) {
nonceKeysToProcIdsMap.remove(nonceKey);
}
}
}
}
@ -224,6 +234,13 @@ public class ProcedureExecutor<TEnvironment> {
private final ConcurrentHashMap<Long, Procedure> procedures =
new ConcurrentHashMap<Long, Procedure>();
/**
* Helper map to lookup whether the procedure already issued from the same client.
* This map contains every root procedure.
*/
private ConcurrentHashMap<NonceKey, Long> nonceKeysToProcIdsMap =
new ConcurrentHashMap<NonceKey, Long>();
/**
* Timeout Queue that contains Procedures in a WAITING_TIMEOUT state
* or periodic procedures.
@ -312,6 +329,11 @@ public class ProcedureExecutor<TEnvironment> {
proc.beforeReplay(getEnvironment());
procedures.put(proc.getProcId(), proc);
// add the nonce to the map
if (proc.getNonceKey() != null) {
nonceKeysToProcIdsMap.put(proc.getNonceKey(), proc.getProcId());
}
if (proc.getState() == ProcedureState.RUNNABLE) {
runnablesCount++;
}
@ -343,6 +365,7 @@ public class ProcedureExecutor<TEnvironment> {
assert !rollbackStack.containsKey(proc.getProcId());
procedures.remove(proc.getProcId());
completed.put(proc.getProcId(), newResultFromProcedure(proc));
continue;
}
@ -479,7 +502,8 @@ public class ProcedureExecutor<TEnvironment> {
}
// Add completed cleaner
waitingTimeout.add(new CompletedProcedureCleaner(conf, store, completed));
waitingTimeout.add(
new CompletedProcedureCleaner(conf, store, completed, nonceKeysToProcIdsMap));
}
public void stop() {
@ -510,6 +534,7 @@ public class ProcedureExecutor<TEnvironment> {
completed.clear();
rollbackStack.clear();
procedures.clear();
nonceKeysToProcIdsMap.clear();
waitingTimeout.clear();
runnables.clear();
lastProcId.set(-1);
@ -552,13 +577,53 @@ public class ProcedureExecutor<TEnvironment> {
* @return the procedure id, that can be used to monitor the operation
*/
public long submitProcedure(final Procedure proc) {
return submitProcedure(proc, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
/**
* Add a new root-procedure to the executor.
* @param proc the new procedure to execute.
* @param nonceGroup
* @param nonce
* @return the procedure id, that can be used to monitor the operation
*/
public long submitProcedure(
final Procedure proc,
final long nonceGroup,
final long nonce) {
Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
Preconditions.checkArgument(isRunning());
Preconditions.checkArgument(lastProcId.get() >= 0);
Preconditions.checkArgument(!proc.hasParent());
// Initialize the Procedure ID
proc.setProcId(nextProcId());
Long currentProcId;
// The following part of the code has to be synchronized to prevent multiple request
// with the same nonce to execute at the same time.
synchronized (this) {
// Check whether the proc exists. If exist, just return the proc id.
// This is to prevent the same proc to submit multiple times (it could happen
// when client could not talk to server and resubmit the same request).
NonceKey noncekey = null;
if (nonce != HConstants.NO_NONCE) {
noncekey = new NonceKey(nonceGroup, nonce);
currentProcId = nonceKeysToProcIdsMap.get(noncekey);
if (currentProcId != null) {
// Found the proc
return currentProcId;
}
}
// Initialize the Procedure ID
currentProcId = nextProcId();
proc.setProcId(currentProcId);
// This is new procedure. Set the noncekey and insert into the map.
if (noncekey != null) {
proc.setNonceKey(noncekey);
nonceKeysToProcIdsMap.put(noncekey, currentProcId);
}
} // end of synchronized (this)
// Commit the transaction
store.insert(proc, null);
@ -568,14 +633,14 @@ public class ProcedureExecutor<TEnvironment> {
// Create the rollback stack for the procedure
RootProcedureState stack = new RootProcedureState();
rollbackStack.put(proc.getProcId(), stack);
rollbackStack.put(currentProcId, stack);
// Submit the new subprocedures
assert !procedures.containsKey(proc.getProcId());
procedures.put(proc.getProcId(), proc);
sendProcedureAddedNotification(proc.getProcId());
assert !procedures.containsKey(currentProcId);
procedures.put(currentProcId, proc);
sendProcedureAddedNotification(currentProcId);
runnables.addBack(proc);
return proc.getProcId();
return currentProcId;
}
public ProcedureResult getResult(final long procId) {
@ -1162,8 +1227,10 @@ public class ProcedureExecutor<TEnvironment> {
private static ProcedureResult newResultFromProcedure(final Procedure proc) {
if (proc.isFailed()) {
return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getException());
return new ProcedureResult(
proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getException());
}
return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getResult());
return new ProcedureResult(
proc.getNonceKey(), proc.getStartTime(), proc.getLastUpdate(), proc.getResult());
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.procedure2;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.NonceKey;
/**
* Once a Procedure completes the ProcedureExecutor takes all the useful
@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ProcedureResult {
private final NonceKey nonceKey;
private final RemoteProcedureException exception;
private final long lastUpdate;
private final long startTime;
@ -37,21 +39,39 @@ public class ProcedureResult {
private long clientAckTime = -1;
public ProcedureResult(final long startTime, final long lastUpdate,
public ProcedureResult(
final NonceKey nonceKey,
final long startTime,
final long lastUpdate,
final RemoteProcedureException exception) {
this.lastUpdate = lastUpdate;
this.startTime = startTime;
this.exception = exception;
this.result = null;
this(nonceKey, exception, lastUpdate, startTime, null);
}
public ProcedureResult(final long startTime, final long lastUpdate, final byte[] result) {
public ProcedureResult(
final NonceKey nonceKey,
final long startTime,
final long lastUpdate,
final byte[] result) {
this(nonceKey, null, lastUpdate, startTime, result);
}
public ProcedureResult(
final NonceKey nonceKey,
final RemoteProcedureException exception,
final long lastUpdate,
final long startTime,
final byte[] result) {
this.nonceKey = nonceKey;
this.exception = exception;
this.lastUpdate = lastUpdate;
this.startTime = startTime;
this.exception = null;
this.result = result;
}
public NonceKey getNonceKey() {
return nonceKey;
}
public boolean isFailed() {
return exception != null;
}

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
@ -119,7 +120,7 @@ public class ProcedureTestingUtility {
procStore.start(1);
procExecutor.start(1, false);
try {
return submitAndWait(procExecutor, proc);
return submitAndWait(procExecutor, proc, HConstants.NO_NONCE, HConstants.NO_NONCE);
} finally {
procStore.stop(false);
procExecutor.stop();
@ -127,7 +128,13 @@ public class ProcedureTestingUtility {
}
public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc) {
long procId = procExecutor.submitProcedure(proc);
return submitAndWait(procExecutor, proc, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc,
final long nonceGroup,
final long nonce) {
long procId = procExecutor.submitProcedure(proc, nonceGroup, nonce);
waitProcedure(procExecutor, procId);
return procId;
}

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
import org.junit.Before;
import org.junit.Assert;
@ -77,6 +76,9 @@ public class TestProcedureRecovery {
procStore.start(PROCEDURE_EXECUTOR_SLOTS);
procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
procSleepInterval = 0;
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, false);
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, false);
}
@After
@ -286,6 +288,41 @@ public class TestProcedureRecovery {
ProcedureTestingUtility.assertIsAbortException(result);
}
@Test(timeout=30000)
public void testCompletedProcWithSameNonce() throws Exception {
final long nonceGroup = 123;
final long nonce = 2222;
Procedure proc = new TestSingleStepProcedure();
// Submit a proc and wait for its completion
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc, nonceGroup, nonce);
// Restart
restart();
Procedure proc2 = new TestSingleStepProcedure();
// Submit a procedure with the same nonce and expect the same procedure would return.
long procId2 = ProcedureTestingUtility.submitAndWait(procExecutor, proc2, nonceGroup, nonce);
assertTrue(procId == procId2);
ProcedureResult result = procExecutor.getResult(procId2);
ProcedureTestingUtility.assertProcNotFailed(result);
}
@Test(timeout=30000)
public void testRunningProcWithSameNonce() throws Exception {
final long nonceGroup = 456;
final long nonce = 33333;
Procedure proc = new TestMultiStepProcedure();
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc, nonceGroup, nonce);
// Restart
restart();
Procedure proc2 = new TestMultiStepProcedure();
// Submit a procedure with the same nonce and expect the same procedure would return.
long procId2 = ProcedureTestingUtility.submitAndWait(procExecutor, proc2, nonceGroup, nonce);
// The original proc is not completed and the new submission should have the same proc Id.
assertTrue(procId == procId2);
}
public static class TestStateMachineProcedure
extends StateMachineProcedure<Void, TestStateMachineProcedure.State> {
enum State { STATE_1, STATE_2, STATE_3, DONE }

View File

@ -382,6 +382,34 @@ public final class ProcedureProtos {
* </pre>
*/
com.google.protobuf.ByteString getStateData();
// optional uint64 nonce_group = 13 [default = 0];
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
boolean hasNonceGroup();
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
long getNonceGroup();
// optional uint64 nonce = 14 [default = 0];
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
boolean hasNonce();
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
long getNonce();
}
/**
* Protobuf type {@code Procedure}
@ -529,6 +557,16 @@ public final class ProcedureProtos {
stateData_ = input.readBytes();
break;
}
case 104: {
bitField0_ |= 0x00000800;
nonceGroup_ = input.readUInt64();
break;
}
case 112: {
bitField0_ |= 0x00001000;
nonce_ = input.readUInt64();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -899,6 +937,46 @@ public final class ProcedureProtos {
return stateData_;
}
// optional uint64 nonce_group = 13 [default = 0];
public static final int NONCE_GROUP_FIELD_NUMBER = 13;
private long nonceGroup_;
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public boolean hasNonceGroup() {
return ((bitField0_ & 0x00000800) == 0x00000800);
}
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public long getNonceGroup() {
return nonceGroup_;
}
// optional uint64 nonce = 14 [default = 0];
public static final int NONCE_FIELD_NUMBER = 14;
private long nonce_;
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public boolean hasNonce() {
return ((bitField0_ & 0x00001000) == 0x00001000);
}
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public long getNonce() {
return nonce_;
}
private void initFields() {
className_ = "";
parentId_ = 0L;
@ -912,6 +990,8 @@ public final class ProcedureProtos {
exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
result_ = com.google.protobuf.ByteString.EMPTY;
stateData_ = com.google.protobuf.ByteString.EMPTY;
nonceGroup_ = 0L;
nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -981,6 +1061,12 @@ public final class ProcedureProtos {
if (((bitField0_ & 0x00000400) == 0x00000400)) {
output.writeBytes(12, stateData_);
}
if (((bitField0_ & 0x00000800) == 0x00000800)) {
output.writeUInt64(13, nonceGroup_);
}
if (((bitField0_ & 0x00001000) == 0x00001000)) {
output.writeUInt64(14, nonce_);
}
getUnknownFields().writeTo(output);
}
@ -1043,6 +1129,14 @@ public final class ProcedureProtos {
size += com.google.protobuf.CodedOutputStream
.computeBytesSize(12, stateData_);
}
if (((bitField0_ & 0x00000800) == 0x00000800)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(13, nonceGroup_);
}
if (((bitField0_ & 0x00001000) == 0x00001000)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(14, nonce_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -1123,6 +1217,16 @@ public final class ProcedureProtos {
result = result && getStateData()
.equals(other.getStateData());
}
result = result && (hasNonceGroup() == other.hasNonceGroup());
if (hasNonceGroup()) {
result = result && (getNonceGroup()
== other.getNonceGroup());
}
result = result && (hasNonce() == other.hasNonce());
if (hasNonce()) {
result = result && (getNonce()
== other.getNonce());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -1184,6 +1288,14 @@ public final class ProcedureProtos {
hash = (37 * hash) + STATE_DATA_FIELD_NUMBER;
hash = (53 * hash) + getStateData().hashCode();
}
if (hasNonceGroup()) {
hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNonceGroup());
}
if (hasNonce()) {
hash = (37 * hash) + NONCE_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNonce());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -1327,6 +1439,10 @@ public final class ProcedureProtos {
bitField0_ = (bitField0_ & ~0x00000400);
stateData_ = com.google.protobuf.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00000800);
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00001000);
nonce_ = 0L;
bitField0_ = (bitField0_ & ~0x00002000);
return this;
}
@ -1408,6 +1524,14 @@ public final class ProcedureProtos {
to_bitField0_ |= 0x00000400;
}
result.stateData_ = stateData_;
if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
to_bitField0_ |= 0x00000800;
}
result.nonceGroup_ = nonceGroup_;
if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
to_bitField0_ |= 0x00001000;
}
result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -1471,6 +1595,12 @@ public final class ProcedureProtos {
if (other.hasStateData()) {
setStateData(other.getStateData());
}
if (other.hasNonceGroup()) {
setNonceGroup(other.getNonceGroup());
}
if (other.hasNonce()) {
setNonce(other.getNonce());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -2274,6 +2404,88 @@ public final class ProcedureProtos {
return this;
}
// optional uint64 nonce_group = 13 [default = 0];
private long nonceGroup_ ;
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public boolean hasNonceGroup() {
return ((bitField0_ & 0x00001000) == 0x00001000);
}
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public long getNonceGroup() {
return nonceGroup_;
}
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public Builder setNonceGroup(long value) {
bitField0_ |= 0x00001000;
nonceGroup_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 nonce_group = 13 [default = 0];</code>
*
* <pre>
* Nonce to prevent same procedure submit by multiple times
* </pre>
*/
public Builder clearNonceGroup() {
bitField0_ = (bitField0_ & ~0x00001000);
nonceGroup_ = 0L;
onChanged();
return this;
}
// optional uint64 nonce = 14 [default = 0];
private long nonce_ ;
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public boolean hasNonce() {
return ((bitField0_ & 0x00002000) == 0x00002000);
}
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public long getNonce() {
return nonce_;
}
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public Builder setNonce(long value) {
bitField0_ |= 0x00002000;
nonce_ = value;
onChanged();
return this;
}
/**
* <code>optional uint64 nonce = 14 [default = 0];</code>
*/
public Builder clearNonce() {
bitField0_ = (bitField0_ & ~0x00002000);
nonce_ = 0L;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:Procedure)
}
@ -7124,33 +7336,34 @@ public final class ProcedureProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
"\n\017Procedure.proto\032\023ErrorHandling.proto\"\217" +
"\n\017Procedure.proto\032\023ErrorHandling.proto\"\271" +
"\002\n\tProcedure\022\022\n\nclass_name\030\001 \002(\t\022\021\n\tpare" +
"nt_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\022\n\nstart_ti" +
"me\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\036\n\005state\030\006 \002(\0162\017" +
".ProcedureState\022\020\n\010stack_id\030\007 \003(\r\022\023\n\013las" +
"t_update\030\010 \002(\004\022\017\n\007timeout\030\t \001(\r\022+\n\texcep" +
"tion\030\n \001(\0132\030.ForeignExceptionMessage\022\016\n\006" +
"result\030\013 \001(\014\022\022\n\nstate_data\030\014 \001(\014\"+\n\027Sequ" +
"entialProcedureData\022\020\n\010executed\030\001 \002(\010\"*\n" +
"\031StateMachineProcedureData\022\r\n\005state\030\001 \003(",
"\r\"X\n\022ProcedureWALHeader\022\017\n\007version\030\001 \002(\r" +
"\022\014\n\004type\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_pr" +
"oc_id\030\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007ve" +
"rsion\030\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\214\001\n\025Pro" +
"cedureStoreTracker\0220\n\004node\030\001 \003(\0132\".Proce" +
"dureStoreTracker.TrackerNode\032A\n\013TrackerN" +
"ode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004\022\017" +
"\n\007deleted\030\003 \003(\004\"\266\001\n\021ProcedureWALEntry\022%\n" +
"\004type\030\001 \002(\0162\027.ProcedureWALEntry.Type\022\035\n\t" +
"procedure\030\002 \003(\0132\n.Procedure\022\017\n\007proc_id\030\003",
" \001(\004\"J\n\004Type\022\007\n\003EOF\020\001\022\010\n\004INIT\020\002\022\n\n\006INSER" +
"T\020\003\022\n\n\006UPDATE\020\004\022\n\n\006DELETE\020\005\022\013\n\007COMPACT\020\006" +
"*p\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" +
"RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIMEO" +
"UT\020\004\022\016\n\nROLLEDBACK\020\005\022\014\n\010FINISHED\020\006BE\n*or" +
"g.apache.hadoop.hbase.protobuf.generated" +
"B\017ProcedureProtosH\001\210\001\001\240\001\001"
"result\030\013 \001(\014\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonc" +
"e_group\030\r \001(\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027Se" +
"quentialProcedureData\022\020\n\010executed\030\001 \002(\010\"",
"*\n\031StateMachineProcedureData\022\r\n\005state\030\001 " +
"\003(\r\"X\n\022ProcedureWALHeader\022\017\n\007version\030\001 \002" +
"(\r\022\014\n\004type\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_" +
"proc_id\030\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007" +
"version\030\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\214\001\n\025P" +
"rocedureStoreTracker\0220\n\004node\030\001 \003(\0132\".Pro" +
"cedureStoreTracker.TrackerNode\032A\n\013Tracke" +
"rNode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004" +
"\022\017\n\007deleted\030\003 \003(\004\"\266\001\n\021ProcedureWALEntry\022" +
"%\n\004type\030\001 \002(\0162\027.ProcedureWALEntry.Type\022\035",
"\n\tprocedure\030\002 \003(\0132\n.Procedure\022\017\n\007proc_id" +
"\030\003 \001(\004\"J\n\004Type\022\007\n\003EOF\020\001\022\010\n\004INIT\020\002\022\n\n\006INS" +
"ERT\020\003\022\n\n\006UPDATE\020\004\022\n\n\006DELETE\020\005\022\013\n\007COMPACT" +
"\020\006*p\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014" +
"\n\010RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIM" +
"EOUT\020\004\022\016\n\nROLLEDBACK\020\005\022\014\n\010FINISHED\020\006BE\n*" +
"org.apache.hadoop.hbase.protobuf.generat" +
"edB\017ProcedureProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -7162,7 +7375,7 @@ public final class ProcedureProtos {
internal_static_Procedure_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_Procedure_descriptor,
new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", });
new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
internal_static_SequentialProcedureData_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_SequentialProcedureData_fieldAccessorTable = new

View File

@ -36,6 +36,8 @@ import "Quota.proto";
message AddColumnRequest {
required TableName table_name = 1;
required ColumnFamilySchema column_families = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message AddColumnResponse {
@ -44,6 +46,8 @@ message AddColumnResponse {
message DeleteColumnRequest {
required TableName table_name = 1;
required bytes column_name = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message DeleteColumnResponse {
@ -52,6 +56,8 @@ message DeleteColumnResponse {
message ModifyColumnRequest {
required TableName table_name = 1;
required ColumnFamilySchema column_families = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message ModifyColumnResponse {
@ -106,6 +112,8 @@ message OfflineRegionResponse {
message CreateTableRequest {
required TableSchema table_schema = 1;
repeated bytes split_keys = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message CreateTableResponse {
@ -114,6 +122,8 @@ message CreateTableResponse {
message DeleteTableRequest {
required TableName table_name = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message DeleteTableResponse {
@ -123,6 +133,8 @@ message DeleteTableResponse {
message TruncateTableRequest {
required TableName tableName = 1;
optional bool preserveSplits = 2 [default = false];
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message TruncateTableResponse {
@ -131,6 +143,8 @@ message TruncateTableResponse {
message EnableTableRequest {
required TableName table_name = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message EnableTableResponse {
@ -139,6 +153,8 @@ message EnableTableResponse {
message DisableTableRequest {
required TableName table_name = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message DisableTableResponse {
@ -148,6 +164,8 @@ message DisableTableResponse {
message ModifyTableRequest {
required TableName table_name = 1;
required TableSchema table_schema = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message ModifyTableResponse {

View File

@ -54,6 +54,10 @@ message Procedure {
optional ForeignExceptionMessage exception = 10;
optional bytes result = 11; // opaque (user) result structure
optional bytes state_data = 12; // opaque (user) procedure internal-state
// Nonce to prevent same procedure submit by multiple times
optional uint64 nonce_group = 13 [default = 0];
optional uint64 nonce = 14 [default = 0];
}
/**

View File

@ -353,7 +353,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
// Disable usage of meta replicas in the master
this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
@ -1402,8 +1402,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long createTable(HTableDescriptor hTableDescriptor,
byte [][] splitKeys) throws IOException {
public long createTable(
final HTableDescriptor hTableDescriptor,
final byte [][] splitKeys,
final long nonceGroup,
final long nonce) throws IOException {
if (isStopped()) {
throw new MasterNotRunningException();
}
@ -1424,8 +1427,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// TableExistsException by saying if the schema is the same or not.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
long procId = this.procedureExecutor.submitProcedure(
new CreateTableProcedure(procedureExecutor.getEnvironment(),
hTableDescriptor, newRegions, latch));
new CreateTableProcedure(
procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch),
nonceGroup,
nonce);
latch.await();
if (cpHost != null) {
@ -1663,7 +1668,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long deleteTable(final TableName tableName) throws IOException {
public long deleteTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
if (cpHost != null) {
cpHost.preDeleteTable(tableName);
@ -1673,7 +1681,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// TODO: We can handle/merge duplicate request
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
long procId = this.procedureExecutor.submitProcedure(
new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch),
nonceGroup,
nonce);
latch.await();
if (cpHost != null) {
@ -1684,7 +1694,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
public long truncateTable(
final TableName tableName,
final boolean preserveSplits,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
if (cpHost != null) {
cpHost.preTruncateTable(tableName);
@ -1692,7 +1706,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
long procId = this.procedureExecutor.submitProcedure(
new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits));
new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
@ -1702,7 +1718,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public void addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor)
public void addColumn(
final TableName tableName,
final HColumnDescriptor columnDescriptor,
final long nonceGroup,
final long nonce)
throws IOException {
checkInitialized();
checkCompression(columnDescriptor);
@ -1713,9 +1733,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, columnDescriptor));
long procId = this.procedureExecutor.submitProcedure(
new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postAddColumn(tableName, columnDescriptor);
@ -1723,7 +1744,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
public void modifyColumn(
final TableName tableName,
final HColumnDescriptor descriptor,
final long nonceGroup,
final long nonce)
throws IOException {
checkInitialized();
checkCompression(descriptor);
@ -1736,9 +1761,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, descriptor));
long procId = this.procedureExecutor.submitProcedure(
new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, descriptor),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
@ -1747,7 +1773,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public void deleteColumn(final TableName tableName, final byte[] columnName)
public void deleteColumn(
final TableName tableName,
final byte[] columnName,
final long nonceGroup,
final long nonce)
throws IOException {
checkInitialized();
if (cpHost != null) {
@ -1758,9 +1788,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, columnName));
long procId = this.procedureExecutor.submitProcedure(
new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnName),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
@ -1769,7 +1800,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long enableTable(final TableName tableName) throws IOException {
public long enableTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
if (cpHost != null) {
cpHost.preEnableTable(tableName);
@ -1778,9 +1812,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Execute the operation asynchronously - client will check the progress of the operation
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
long procId =
this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor
.getEnvironment(), tableName, false, prepareLatch));
long procId = this.procedureExecutor.submitProcedure(
new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch),
nonceGroup,
nonce);
// Before returning to client, we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
//
@ -1795,7 +1830,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long disableTable(final TableName tableName) throws IOException {
public long disableTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
if (cpHost != null) {
cpHost.preDisableTable(tableName);
@ -1805,9 +1843,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Execute the operation asynchronously - client will check the progress of the operation
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
// Execute the operation asynchronously - client will check the progress of the operation
long procId =
this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor
.getEnvironment(), tableName, false, prepareLatch));
long procId = this.procedureExecutor.submitProcedure(
new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch),
nonceGroup,
nonce);
// Before returning to client, we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
//
@ -1857,7 +1896,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public long modifyTable(final TableName tableName, final HTableDescriptor descriptor)
public long modifyTable(
final TableName tableName,
final HTableDescriptor descriptor,
final long nonceGroup,
final long nonce)
throws IOException {
checkInitialized();
sanityCheckTableDescriptor(descriptor);
@ -1869,7 +1912,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Execute the operation synchronously - wait for the operation completes before continuing.
long procId = this.procedureExecutor.submitProcedure(
new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor));
new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);

View File

@ -348,8 +348,11 @@ public class MasterRpcServices extends RSRpcServices
public AddColumnResponse addColumn(RpcController controller,
AddColumnRequest req) throws ServiceException {
try {
master.addColumn(ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()));
master.addColumn(
ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()),
req.getNonceGroup(),
req.getNonce());
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
@ -416,7 +419,8 @@ public class MasterRpcServices extends RSRpcServices
HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
try {
long procId = master.createTable(hTableDescriptor, splitKeys);
long procId =
master.createTable(hTableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
return CreateTableResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
@ -427,8 +431,11 @@ public class MasterRpcServices extends RSRpcServices
public DeleteColumnResponse deleteColumn(RpcController controller,
DeleteColumnRequest req) throws ServiceException {
try {
master.deleteColumn(ProtobufUtil.toTableName(req.getTableName()),
req.getColumnName().toByteArray());
master.deleteColumn(
ProtobufUtil.toTableName(req.getTableName()),
req.getColumnName().toByteArray(),
req.getNonceGroup(),
req.getNonce());
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
@ -472,7 +479,8 @@ public class MasterRpcServices extends RSRpcServices
public DeleteTableResponse deleteTable(RpcController controller,
DeleteTableRequest request) throws ServiceException {
try {
long procId = master.deleteTable(ProtobufUtil.toTableName(request.getTableName()));
long procId = master.deleteTable(ProtobufUtil.toTableName(
request.getTableName()), request.getNonceGroup(), request.getNonce());
return DeleteTableResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
@ -483,9 +491,11 @@ public class MasterRpcServices extends RSRpcServices
public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
throws ServiceException {
try {
long procId =
master.truncateTable(ProtobufUtil.toTableName(request.getTableName()),
request.getPreserveSplits());
long procId = master.truncateTable(
ProtobufUtil.toTableName(request.getTableName()),
request.getPreserveSplits(),
request.getNonceGroup(),
request.getNonce());
return TruncateTableResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
@ -496,7 +506,10 @@ public class MasterRpcServices extends RSRpcServices
public DisableTableResponse disableTable(RpcController controller,
DisableTableRequest request) throws ServiceException {
try {
long procId = master.disableTable(ProtobufUtil.toTableName(request.getTableName()));
long procId = master.disableTable(
ProtobufUtil.toTableName(request.getTableName()),
request.getNonceGroup(),
request.getNonce());
return DisableTableResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
@ -582,7 +595,10 @@ public class MasterRpcServices extends RSRpcServices
public EnableTableResponse enableTable(RpcController controller,
EnableTableRequest request) throws ServiceException {
try {
long procId = master.enableTable(ProtobufUtil.toTableName(request.getTableName()));
long procId = master.enableTable(
ProtobufUtil.toTableName(request.getTableName()),
request.getNonceGroup(),
request.getNonce());
return EnableTableResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
@ -1069,8 +1085,11 @@ public class MasterRpcServices extends RSRpcServices
public ModifyColumnResponse modifyColumn(RpcController controller,
ModifyColumnRequest req) throws ServiceException {
try {
master.modifyColumn(ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()));
master.modifyColumn(
ProtobufUtil.toTableName(req.getTableName()),
HColumnDescriptor.convert(req.getColumnFamilies()),
req.getNonceGroup(),
req.getNonce());
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
@ -1093,8 +1112,11 @@ public class MasterRpcServices extends RSRpcServices
public ModifyTableResponse modifyTable(RpcController controller,
ModifyTableRequest req) throws ServiceException {
try {
master.modifyTable(ProtobufUtil.toTableName(req.getTableName()),
HTableDescriptor.convert(req.getTableSchema()));
master.modifyTable(
ProtobufUtil.toTableName(req.getTableName()),
HTableDescriptor.convert(req.getTableSchema()),
req.getNonceGroup(),
req.getNonce());
} catch (IOException ioe) {
throw new ServiceException(ioe);
}

View File

@ -103,75 +103,125 @@ public interface MasterServices extends Server {
* Create a table using the given table definition.
* @param desc The table definition
* @param splitKeys Starting row keys for the initial table regions. If null
* @param nonceGroup
* @param nonce
* a single region is created.
*/
long createTable(HTableDescriptor desc, byte[][] splitKeys)
throws IOException;
long createTable(
final HTableDescriptor desc,
final byte[][] splitKeys,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Delete a table
* @param tableName The table name
* @param nonceGroup
* @param nonce
* @throws IOException
*/
long deleteTable(final TableName tableName) throws IOException;
long deleteTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Truncate a table
* @param tableName The table name
* @param preserveSplits True if the splits should be preserved
* @param nonceGroup
* @param nonce
* @throws IOException
*/
public long truncateTable(final TableName tableName, boolean preserveSplits) throws IOException;
public long truncateTable(
final TableName tableName,
final boolean preserveSplits,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Modify the descriptor of an existing table
* @param tableName The table name
* @param descriptor The updated table descriptor
* @param nonceGroup
* @param nonce
* @throws IOException
*/
long modifyTable(final TableName tableName, final HTableDescriptor descriptor)
long modifyTable(
final TableName tableName,
final HTableDescriptor descriptor,
final long nonceGroup,
final long nonce)
throws IOException;
/**
* Enable an existing table
* @param tableName The table name
* @param nonceGroup
* @param nonce
* @throws IOException
*/
long enableTable(final TableName tableName) throws IOException;
long enableTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Disable an existing table
* @param tableName The table name
* @param nonceGroup
* @param nonce
* @throws IOException
*/
long disableTable(final TableName tableName) throws IOException;
long disableTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Add a new column to an existing table
* @param tableName The table name
* @param column The column definition
* @param nonceGroup
* @param nonce
* @throws IOException
*/
void addColumn(final TableName tableName, final HColumnDescriptor column)
void addColumn(
final TableName tableName,
final HColumnDescriptor column,
final long nonceGroup,
final long nonce)
throws IOException;
/**
* Modify the column descriptor of an existing column in an existing table
* @param tableName The table name
* @param descriptor The updated column definition
* @param nonceGroup
* @param nonce
* @throws IOException
*/
void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
void modifyColumn(
final TableName tableName,
final HColumnDescriptor descriptor,
final long nonceGroup,
final long nonce)
throws IOException;
/**
* Delete a column from an existing table
* @param tableName The table name
* @param columnName The column name
* @param nonceGroup
* @param nonce
* @throws IOException
*/
void deleteColumn(final TableName tableName, final byte[] columnName)
void deleteColumn(
final TableName tableName,
final byte[] columnName,
final long nonceGroup,
final long nonce)
throws IOException;
/**

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.NonceKey;
import com.google.common.annotations.VisibleForTesting;
@ -102,38 +103,6 @@ public class ServerNonceManager {
}
}
/**
* This implementation is not smart and just treats nonce group and nonce as random bits.
*/
// TODO: we could use pure byte arrays, but then we wouldn't be able to use hash map.
private static class NonceKey {
private long group;
private long nonce;
public NonceKey(long group, long nonce) {
assert nonce != HConstants.NO_NONCE;
this.group = group;
this.nonce = nonce;
}
@Override
public boolean equals(Object obj) {
if (obj == null || !(obj instanceof NonceKey)) return false;
NonceKey nk = ((NonceKey)obj);
return this.nonce == nk.nonce && this.group == nk.group;
}
@Override
public int hashCode() {
return (int)((group >> 32) ^ group ^ (nonce >> 32) ^ nonce);
}
@Override
public String toString() {
return "[" + group + ":" + nonce + "]";
}
}
/**
* Nonces.
* Approximate overhead per nonce: 64 bytes from hashmap, 32 from two objects (k/v),

View File

@ -136,7 +136,9 @@ public class AccessControlLists {
// Set cache data blocks in L1 if more than one cache tier deployed; e.g. this will
// be the case if we are using CombinedBlockCache (Bucket Cache).
.setCacheDataInL1(true)),
null);
null,
HConstants.NO_NONCE,
HConstants.NO_NONCE);
}
/**

View File

@ -198,7 +198,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
DisabledRegionSplitPolicy.class.getName());
labelsTable.setValue(Bytes.toBytes(HConstants.DISALLOW_WRITES_IN_RECOVERING),
Bytes.toBytes(true));
master.createTable(labelsTable, null);
master.createTable(labelsTable, null, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Matchers;
@ -67,6 +68,10 @@ public class TestHBaseAdminNoCluster {
* @throws MasterNotRunningException
* @throws ServiceException
*/
//TODO: Clean up, with Procedure V2 and nonce to prevent the same procedure to call mulitple
// time, this test is invalid anymore. Just keep the test around for some time before
// fully removing it.
@Ignore
@Test
public void testMasterMonitorCallableRetries()
throws MasterNotRunningException, ZooKeeperConnectionException, IOException, ServiceException {

View File

@ -1056,7 +1056,7 @@ public class TestAssignmentManagerOnCluster {
assertEquals(oldServerName, regionStates.getRegionServerOfRegion(hri));
// Disable the table now.
master.disableTable(hri.getTable());
master.disableTable(hri.getTable(), HConstants.NO_NONCE, HConstants.NO_NONCE);
// Kill the hosting server, which doesn't have meta on it.
cluster.killRegionServer(oldServerName);

View File

@ -226,8 +226,11 @@ public class TestCatalogJanitor {
}
@Override
public long createTable(HTableDescriptor desc, byte[][] splitKeys)
throws IOException {
public long createTable(
final HTableDescriptor desc,
final byte[][] splitKeys,
final long nonceGroup,
final long nonce) throws IOException {
// no-op
return -1;
}
@ -427,43 +430,68 @@ public class TestCatalogJanitor {
}
@Override
public long deleteTable(TableName tableName) throws IOException {
public long deleteTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
public long truncateTable(
final TableName tableName,
final boolean preserveSplits,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public long modifyTable(TableName tableName, HTableDescriptor descriptor)
throws IOException {
public long modifyTable(
final TableName tableName,
final HTableDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public long enableTable(TableName tableName) throws IOException {
public long enableTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public long disableTable(TableName tableName) throws IOException {
public long disableTable(
TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public void addColumn(TableName tableName, HColumnDescriptor column)
throws IOException { }
public void addColumn(
final TableName tableName,
final HColumnDescriptor columnDescriptor,
final long nonceGroup,
final long nonce) throws IOException { }
@Override
public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
throws IOException { }
public void modifyColumn(
final TableName tableName,
final HColumnDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException { }
@Override
public void deleteColumn(TableName tableName, byte[] columnName)
throws IOException { }
public void deleteColumn(
final TableName tableName,
final byte[] columnName,
final long nonceGroup,
final long nonce) throws IOException { }
@Override
public TableLockManager getTableLockManager() {

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
@ -426,6 +427,14 @@ public class MasterProcedureTestingUtility {
return put;
}
public static long generateNonceGroup(final HMaster master) {
return master.getConnection().getNonceGenerator().getNonceGroup();
}
public static long generateNonce(final HMaster master) {
return master.getConnection().getNonceGenerator().newNonce();
}
public static class InjectAbortOnLoadListener
implements ProcedureExecutor.ProcedureExecutorListener {
private final ProcedureExecutor<MasterProcedureEnv> procExec;

View File

@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
@ -47,6 +48,9 @@ public class TestAddColumnFamilyProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -69,6 +73,9 @@ public class TestAddColumnFamilyProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -92,9 +99,10 @@ public class TestAddColumnFamilyProcedure {
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f3");
// Test 1: Add a column family online
long procId1 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor1));
long procId1 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor1),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
@ -104,9 +112,10 @@ public class TestAddColumnFamilyProcedure {
// Test 2: Add a column family offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId2 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor2));
long procId2 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
@ -125,9 +134,10 @@ public class TestAddColumnFamilyProcedure {
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
// add the column family
long procId1 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId1 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
@ -135,9 +145,10 @@ public class TestAddColumnFamilyProcedure {
tableName, cf2);
// add the column family that exists
long procId2 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId2 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
@ -149,9 +160,10 @@ public class TestAddColumnFamilyProcedure {
// Do the same add the existing column family - this time offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId3 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId3 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup + 2,
nonce + 2);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId3);
@ -162,6 +174,37 @@ public class TestAddColumnFamilyProcedure {
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testAddSameColumnFamilyTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testAddSameColumnFamilyTwiceWithSameNonce");
final String cf2 = "cf2";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
// add the column family
long procId1 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
long procId2 = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf2);
// Wait the completion and expect not fail - because it is the same proc
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
@ -176,9 +219,10 @@ public class TestAddColumnFamilyProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = AddColumnFamilyState.values().length;
@ -202,9 +246,10 @@ public class TestAddColumnFamilyProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = AddColumnFamilyState.values().length;
@ -228,9 +273,10 @@ public class TestAddColumnFamilyProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
long procId = procExec.submitProcedure(
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
int numberOfSteps = AddColumnFamilyState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,

View File

@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -43,10 +43,7 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MasterTests.class, MediumTests.class})
public class TestCreateTableProcedure {
@ -54,6 +51,9 @@ public class TestCreateTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -76,6 +76,9 @@ public class TestCreateTableProcedure {
@Before
public void setup() throws Exception {
resetProcExecutorTestingKillFlag();
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -125,12 +128,14 @@ public class TestCreateTableProcedure {
// create the table
long procId1 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
// create another with the same name
ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch();
long procId2 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2));
new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2),
nonceGroup + 1,
nonce + 1);
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
@ -139,6 +144,29 @@ public class TestCreateTableProcedure {
latch2.await();
}
@Test(timeout=60000)
public void testCreateTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testCreateTwiceWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
// create the table
long procId1 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
// create another with the same name
long procId2 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
assertTrue(procId1 == procId2);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
@ -152,7 +180,7 @@ public class TestCreateTableProcedure {
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
// Restart the executor and execute the step twice
// NOTE: the 6 (number of CreateTableState steps) is hardcoded,
@ -180,7 +208,7 @@ public class TestCreateTableProcedure {
htd.setRegionReplication(3);
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
// NOTE: the 4 (number of CreateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
@ -210,7 +238,7 @@ public class TestCreateTableProcedure {
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions));
new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
// NOTE: the 4 (number of CreateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.

View File

@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@ -47,6 +48,9 @@ public class TestDeleteColumnFamilyProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -69,6 +73,9 @@ public class TestDeleteColumnFamilyProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -90,9 +97,10 @@ public class TestDeleteColumnFamilyProcedure {
MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, "f3");
// Test 1: delete the column family that exists online
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf1.getBytes()));
long procId1 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf1.getBytes()),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
@ -102,9 +110,10 @@ public class TestDeleteColumnFamilyProcedure {
// Test 2: delete the column family that exists offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId2 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
long procId2 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
@ -120,9 +129,10 @@ public class TestDeleteColumnFamilyProcedure {
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2);
// delete the column family that exists
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
long procId1 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
// First delete should succeed
@ -132,9 +142,10 @@ public class TestDeleteColumnFamilyProcedure {
tableName, cf2);
// delete the column family that does not exist
long procId2 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
long procId2 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
@ -147,9 +158,10 @@ public class TestDeleteColumnFamilyProcedure {
// Try again, this time with table disabled.
UTIL.getHBaseAdmin().disableTable(tableName);
long procId3 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
long procId3 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup + 2,
nonce + 2);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId3);
// Expect fail with InvalidFamilyOperationException
@ -159,6 +171,37 @@ public class TestDeleteColumnFamilyProcedure {
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testDeleteColumnFamilyTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteColumnFamilyTwiceWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final String cf2 = "cf2";
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2);
// delete the column family that exists
long procId1 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup,
nonce);
long procId2 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf2);
// Wait the completion and expect not fail - because it is the same proc
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout=60000)
public void testDeleteNonExistingColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily");
@ -169,9 +212,10 @@ public class TestDeleteColumnFamilyProcedure {
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// delete the column family that does not exist
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf3.getBytes()));
long procId1 = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf3.getBytes()),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
@ -195,9 +239,10 @@ public class TestDeleteColumnFamilyProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf4.getBytes()));
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = DeleteColumnFamilyState.values().length;
@ -221,9 +266,10 @@ public class TestDeleteColumnFamilyProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf5.getBytes()));
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = DeleteColumnFamilyState.values().length;
@ -249,7 +295,9 @@ public class TestDeleteColumnFamilyProcedure {
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()),
nonceGroup,
nonce);
// Failing before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should trigger the rollback
// NOTE: the 1 (number before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT step) is hardcoded,
@ -280,7 +328,9 @@ public class TestDeleteColumnFamilyProcedure {
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()),
nonceGroup,
nonce);
// Failing after DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should not trigger the rollback.
// NOTE: the 4 (number of DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT + 1 step) is hardcoded,

View File

@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTa
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -42,10 +42,7 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MasterTests.class, MediumTests.class})
public class TestDeleteTableProcedure {
@ -53,6 +50,9 @@ public class TestDeleteTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private long nonceGroup = HConstants.NO_NONCE;
private long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -77,6 +77,10 @@ public class TestDeleteTableProcedure {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
assertTrue("expected executor to be running", procExec.isRunning());
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -123,10 +127,10 @@ public class TestDeleteTableProcedure {
// delete the table (that exists)
long procId1 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
// delete the table (that will no longer exist)
long procId2 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup + 1, nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
@ -144,6 +148,36 @@ public class TestDeleteTableProcedure {
assertTrue(result.getException().getCause() instanceof TableNotFoundException);
}
@Test(timeout=60000)
public void testDoubleDeletedTableWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testDoubleDeletedTableWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f");
UTIL.getHBaseAdmin().disableTable(tableName);
// delete the table (that exists)
long procId1 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
// delete the table (that will no longer exist)
long procId2 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// First delete should succeed
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
// Second delete should not fail, because it is the same delete
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout=60000)
public void testSimpleDelete() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleDelete");
@ -190,7 +224,7 @@ public class TestDeleteTableProcedure {
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
// Restart the executor and execute the step twice
// NOTE: the 6 (number of DeleteTableState steps) is hardcoded,

View File

@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableT
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -49,6 +49,9 @@ public class TestDisableTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -71,6 +74,9 @@ public class TestDisableTableProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -91,7 +97,7 @@ public class TestDisableTableProcedure {
// Disable the table
long procId = procExec.submitProcedure(
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
@ -108,7 +114,7 @@ public class TestDisableTableProcedure {
// Disable the table
long procId1 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false));
procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
@ -117,7 +123,7 @@ public class TestDisableTableProcedure {
// Disable the table again - expect failure
long procId2 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false));
procExec.getEnvironment(), tableName, false), nonceGroup + 1, nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureResult result = procExec.getResult(procId2);
@ -130,7 +136,7 @@ public class TestDisableTableProcedure {
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId3 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false, prepareLatch));
procExec.getEnvironment(), tableName, false, prepareLatch), nonceGroup + 2, nonce + 2);
prepareLatch.await();
Assert.fail("Disable should throw exception through latch.");
} catch (TableNotEnabledException tnee) {
@ -148,6 +154,29 @@ public class TestDisableTableProcedure {
tableName);
}
@Test(timeout = 60000)
public void testDisableTableTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testDisableTableTwiceWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// Disable the table
long procId1 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
long procId2 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
@ -161,9 +190,8 @@ public class TestDisableTableProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Disable procedure && kill the executor
long procId =
procExec.submitProcedure(new DisableTableProcedure(procExec.getEnvironment(), tableName,
false));
long procId = procExec.submitProcedure(
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Restart the executor and execute the step twice
int numberOfSteps = DisableTableState.values().length;

View File

@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTa
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -49,6 +49,9 @@ public class TestEnableTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -71,6 +74,9 @@ public class TestEnableTableProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -92,7 +98,7 @@ public class TestEnableTableProcedure {
// Enable the table
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
@ -100,6 +106,29 @@ public class TestEnableTableProcedure {
tableName);
}
@Test(timeout = 60000)
public void testEnableTableTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testEnableTableTwiceWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
// Enable the table
long procId1 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
long procId2 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
// The second proc should succeed too - because it is the same proc.
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout=60000, expected=TableNotDisabledException.class)
public void testEnableNonDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf("testEnableNonExistingTable");
@ -109,7 +138,7 @@ public class TestEnableTableProcedure {
// Enable the table - expect failure
long procId1 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
@ -119,7 +148,9 @@ public class TestEnableTableProcedure {
// Enable the table with skipping table state check flag (simulate recovery scenario)
long procId2 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, true));
new EnableTableProcedure(procExec.getEnvironment(), tableName, true),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
@ -127,7 +158,9 @@ public class TestEnableTableProcedure {
// Enable the table - expect failure from ProcedurePrepareLatch
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId3 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch));
new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch),
nonceGroup + 2,
nonce + 2);
prepareLatch.await();
Assert.fail("Enable should throw exception through latch.");
}
@ -147,7 +180,7 @@ public class TestEnableTableProcedure {
// Start the Enable procedure && kill the executor
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
// Restart the executor and execute the step twice
int numberOfSteps = EnableTableState.values().length;
@ -175,7 +208,7 @@ public class TestEnableTableProcedure {
// Start the Enable procedure && kill the executor
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
int numberOfSteps = EnableTableState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(

View File

@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
@ -47,6 +48,9 @@ public class TestModifyColumnFamilyProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -69,6 +73,9 @@ public class TestModifyColumnFamilyProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -94,8 +101,10 @@ public class TestModifyColumnFamilyProcedure {
// Test 1: modify the column family online
columnDescriptor.setBlocksize(newBlockSize);
long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
long procId1 = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
@ -105,9 +114,10 @@ public class TestModifyColumnFamilyProcedure {
// Test 2: modify the column family offline
UTIL.getHBaseAdmin().disableTable(tableName);
columnDescriptor.setBlocksize(newBlockSize * 2);
long procId2 =
procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
tableName, columnDescriptor));
long procId2 = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
@ -129,8 +139,10 @@ public class TestModifyColumnFamilyProcedure {
// Modify the column family that does not exist
columnDescriptor.setBlocksize(newBlockSize);
long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
long procId1 = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
@ -158,8 +170,10 @@ public class TestModifyColumnFamilyProcedure {
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
long procId = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = ModifyColumnFamilyState.values().length;
@ -190,9 +204,10 @@ public class TestModifyColumnFamilyProcedure {
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId =
procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
tableName, columnDescriptor));
long procId = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = ModifyColumnFamilyState.values().length;
@ -220,8 +235,10 @@ public class TestModifyColumnFamilyProcedure {
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
long procId = procExec.submitProcedure(
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
nonceGroup,
nonce);
// Failing in the middle of proc
int numberOfSteps = ModifyColumnFamilyState.values().length - 2;

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
@ -48,6 +49,9 @@ public class TestModifyTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -70,6 +74,9 @@ public class TestModifyTableProcedure {
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -224,8 +231,8 @@ public class TestModifyTableProcedure {
htd.setRegionReplication(3);
// Start the Modify procedure && kill the executor
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
// Restart the executor and execute the step twice
int numberOfSteps = ModifyTableState.values().length;
@ -266,8 +273,8 @@ public class TestModifyTableProcedure {
htd.removeFamily(cf3.getBytes());
// Start the Modify procedure && kill the executor
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
// Restart the executor and execute the step twice
int numberOfSteps = ModifyTableState.values().length;
@ -304,8 +311,8 @@ public class TestModifyTableProcedure {
htd.addFamily(new HColumnDescriptor(familyName));
// Start the Modify procedure && kill the executor
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
// Restart the executor and rollback the step twice
int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc
@ -341,8 +348,8 @@ public class TestModifyTableProcedure {
htd.setRegionReplication(3);
// Start the Modify procedure && kill the executor
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
// Restart the executor and rollback the step twice
int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc
@ -379,8 +386,8 @@ public class TestModifyTableProcedure {
htd.setRegionReplication(3);
// Start the Modify procedure && kill the executor
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
// Failing after MODIFY_TABLE_DELETE_FS_LAYOUT we should not trigger the rollback.
// NOTE: the 5 (number of MODIFY_TABLE_DELETE_FS_LAYOUT + 1 step) is hardcoded,

View File

@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.Truncate
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -43,9 +43,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MasterTests.class, MediumTests.class})
public class TestTruncateTableProcedure {
@ -53,6 +51,9 @@ public class TestTruncateTableProcedure {
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@ -77,6 +78,10 @@ public class TestTruncateTableProcedure {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
assertTrue("expected executor to be running", procExec.isRunning());
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
@ -210,7 +215,9 @@ public class TestTruncateTableProcedure {
// Start the Truncate procedure && kill the executor
long procId = procExec.submitProcedure(
new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
// NOTE: the 7 (number of TruncateTableState steps) is hardcoded,