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:
parent
60d6435ad8
commit
951ec7a0b7
|
@ -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);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
||||
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
|
||||
proc.setProcId(nextProcId());
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 }
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
Loading…
Reference in New Issue