HBASE-21278 Do not rollback successful sub procedures when rolling back a procedure

This commit is contained in:
zhangduo 2018-10-14 17:13:34 +08:00 committed by Duo Zhang
parent c9dcc9a065
commit 0d9982901a
15 changed files with 186 additions and 129 deletions

View File

@ -1594,7 +1594,19 @@ public class ProcedureExecutor<TEnvironment> {
int stackTail = subprocStack.size();
while (stackTail-- > 0) {
Procedure<TEnvironment> proc = subprocStack.get(stackTail);
// For the sub procedures which are successfully finished, we do not rollback them.
// Typically, if we want to rollback a procedure, we first need to rollback it, and then
// recursively rollback its ancestors. The state changes which are done by sub procedures
// should be handled by parent procedures when rolling back. For example, when rolling back a
// MergeTableProcedure, we will schedule new procedures to bring the offline regions online,
// instead of rolling back the original procedures which offlined the regions(in fact these
// procedures can not be rolled back...).
if (proc.isSuccess()) {
// Just do the cleanup work, without actually executing the rollback
subprocStack.remove(stackTail);
cleanupAfterRollbackOneStep(proc);
continue;
}
LockState lockState = acquireLock(proc);
if (lockState != LockState.LOCK_ACQUIRED) {
// can't take a lock on the procedure, add the root-proc back on the
@ -1633,6 +1645,31 @@ public class ProcedureExecutor<TEnvironment> {
return LockState.LOCK_ACQUIRED;
}
private void cleanupAfterRollbackOneStep(Procedure<TEnvironment> proc) {
if (proc.removeStackIndex()) {
if (!proc.isSuccess()) {
proc.setState(ProcedureState.ROLLEDBACK);
}
// update metrics on finishing the procedure (fail)
proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false);
if (proc.hasParent()) {
store.delete(proc.getProcId());
procedures.remove(proc.getProcId());
} else {
final long[] childProcIds = rollbackStack.get(proc.getProcId()).getSubprocedureIds();
if (childProcIds != null) {
store.delete(proc, childProcIds);
} else {
store.update(proc);
}
}
} else {
store.update(proc);
}
}
/**
* Execute the rollback of the procedure step.
* It updates the store with the new state (stack index)
@ -1661,26 +1698,7 @@ public class ProcedureExecutor<TEnvironment> {
throw new RuntimeException(msg);
}
if (proc.removeStackIndex()) {
proc.setState(ProcedureState.ROLLEDBACK);
// update metrics on finishing the procedure (fail)
proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false);
if (proc.hasParent()) {
store.delete(proc.getProcId());
procedures.remove(proc.getProcId());
} else {
final long[] childProcIds = rollbackStack.get(proc.getProcId()).getSubprocedureIds();
if (childProcIds != null) {
store.delete(proc, childProcIds);
} else {
store.update(proc);
}
}
} else {
store.update(proc);
}
cleanupAfterRollbackOneStep(proc);
return LockState.LOCK_ACQUIRED;
}

View File

@ -219,12 +219,16 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
@Override
protected boolean abort(final TEnvironment env) {
LOG.debug("Abort requested for {}", this);
if (hasMoreState()) {
aborted.set(true);
return true;
if (!hasMoreState()) {
LOG.warn("Ignore abort request on {} because it has already been finished", this);
return false;
}
LOG.debug("Ignoring abort request on {}", this);
return false;
if (!isRollbackSupported(getCurrentState())) {
LOG.warn("Ignore abort request on {} because it does not support rollback", this);
return false;
}
aborted.set(true);
return true;
}
/**

View File

@ -52,7 +52,7 @@ public class TestProcedureExecution {
private static final Logger LOG = LoggerFactory.getLogger(TestProcedureExecution.class);
private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
private static final Procedure NULL_PROC = null;
private static final Procedure<?> NULL_PROC = null;
private ProcedureExecutor<Void> procExecutor;
private ProcedureStore procStore;
@ -84,11 +84,16 @@ public class TestProcedureExecution {
}
private static class TestProcedureException extends IOException {
public TestProcedureException(String msg) { super(msg); }
private static final long serialVersionUID = 8798565784658913798L;
public TestProcedureException(String msg) {
super(msg);
}
}
public static class TestSequentialProcedure extends SequentialProcedure<Void> {
private final Procedure[] subProcs;
private final Procedure<Void>[] subProcs;
private final List<String> state;
private final Exception failure;
private final String name;
@ -112,7 +117,7 @@ public class TestProcedureExecution {
}
@Override
protected Procedure[] execute(Void env) {
protected Procedure<Void>[] execute(Void env) {
state.add(name + "-execute");
if (failure != null) {
setFailure(new RemoteProcedureException(name + "-failure", failure));
@ -136,9 +141,9 @@ public class TestProcedureExecution {
@Test
public void testBadSubprocList() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2, NULL_PROC);
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
Procedure<Void> subProc2 = new TestSequentialProcedure("subProc2", state);
Procedure<Void> subProc1 = new TestSequentialProcedure("subProc1", state, subProc2, NULL_PROC);
Procedure<Void> rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
// subProc1 has a "null" subprocedure which is catched as InvalidArgument
@ -158,9 +163,9 @@ public class TestProcedureExecution {
@Test
public void testSingleSequentialProc() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
Procedure<Void> subProc2 = new TestSequentialProcedure("subProc2", state);
Procedure<Void> subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
Procedure<Void> rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
// successful state, with 3 execute
@ -173,10 +178,10 @@ public class TestProcedureExecution {
@Test
public void testSingleSequentialProcRollback() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state,
new TestProcedureException("fail test"));
Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
Procedure<Void> subProc2 =
new TestSequentialProcedure("subProc2", state, new TestProcedureException("fail test"));
Procedure<Void> subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
Procedure<Void> rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
// the 3rd proc fail, rollback after 2 successful execution
@ -203,7 +208,7 @@ public class TestProcedureExecution {
public TestFaultyRollback() { }
@Override
protected Procedure[] execute(Void env) {
protected Procedure<Void>[] execute(Void env) {
setFailure("faulty-rollback-test", new TestProcedureException("test faulty rollback"));
return null;
}
@ -249,7 +254,7 @@ public class TestProcedureExecution {
}
@Override
protected Procedure[] execute(Void env) {
protected Procedure<Void>[] execute(Void env) {
state.add(name + "-execute");
setState(ProcedureState.WAITING_TIMEOUT);
return hasChild ? new Procedure[] { new TestWaitChild(name, state) } : null;
@ -280,14 +285,14 @@ public class TestProcedureExecution {
}
@Override
protected Procedure[] execute(Void env) {
protected Procedure<Void>[] execute(Void env) {
state.add(name + "-child-execute");
return null;
}
@Override
protected void rollback(Void env) {
state.add(name + "-child-rollback");
throw new UnsupportedOperationException("should not rollback a successful child procedure");
}
@Override
@ -302,7 +307,7 @@ public class TestProcedureExecution {
public void testAbortTimeout() {
final int PROC_TIMEOUT_MSEC = 2500;
List<String> state = new ArrayList<>();
Procedure proc = new TestWaitingProcedure("wproc", state, false);
Procedure<Void> proc = new TestWaitingProcedure("wproc", state, false);
proc.setTimeout(PROC_TIMEOUT_MSEC);
long startTime = EnvironmentEdgeManager.currentTime();
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
@ -320,17 +325,16 @@ public class TestProcedureExecution {
@Test
public void testAbortTimeoutWithChildren() {
List<String> state = new ArrayList<>();
Procedure proc = new TestWaitingProcedure("wproc", state, true);
Procedure<Void> proc = new TestWaitingProcedure("wproc", state, true);
proc.setTimeout(2500);
long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
LOG.info(Objects.toString(state));
Procedure<?> result = procExecutor.getResult(rootId);
assertTrue(state.toString(), result.isFailed());
ProcedureTestingUtility.assertIsTimeoutException(result);
assertEquals(state.toString(), 4, state.size());
assertEquals(state.toString(), 3, state.size());
assertEquals("wproc-execute", state.get(0));
assertEquals("wproc-child-execute", state.get(1));
assertEquals("wproc-child-rollback", state.get(2));
assertEquals("wproc-rollback", state.get(3));
assertEquals("wproc-rollback", state.get(2));
}
}

View File

@ -48,6 +48,9 @@ public class TestStateMachineProcedure {
private static final Logger LOG = LoggerFactory.getLogger(TestStateMachineProcedure.class);
private static final Exception TEST_FAILURE_EXCEPTION = new Exception("test failure") {
private static final long serialVersionUID = 2147942238987041310L;
@Override
public boolean equals(final Object other) {
if (this == other) return true;
@ -192,6 +195,11 @@ public class TestStateMachineProcedure {
return Flow.HAS_MORE_STATE;
}
@Override
protected boolean isRollbackSupported(TestSMProcedureState state) {
return true;
}
@Override
protected void rollbackState(TestProcEnv env, TestSMProcedureState state) {
LOG.info("ROLLBACK " + state + " " + this);
@ -274,7 +282,7 @@ public class TestStateMachineProcedure {
public static class SimpleChildProcedure extends NoopProcedure<TestProcEnv> {
@Override
protected Procedure[] execute(TestProcEnv env) {
protected Procedure<TestProcEnv>[] execute(TestProcEnv env) {
LOG.info("EXEC " + this);
env.execCount.incrementAndGet();
if (env.triggerChildRollback) {

View File

@ -140,26 +140,45 @@ final class AssignmentManagerUtil {
return procs;
}
/**
* Create assign procedures for the give regions, according to the {@code regionReplication}.
* <p/>
* For rolling back, we will submit procedures directly to the {@code ProcedureExecutor}, so it is
* possible that we persist the newly scheduled procedures, and then crash before persisting the
* rollback state, so when we arrive here the second time, it is possible that some regions have
* already been associated with a TRSP.
* @param ignoreIfInTransition if true, will skip creating TRSP for the given region if it is
* already in transition, otherwise we will add an assert that it should not in
* transition.
*/
private static TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env,
List<RegionInfo> regions, int regionReplication, ServerName targetServer) {
List<RegionInfo> regions, int regionReplication, ServerName targetServer,
boolean ignoreIfInTransition) {
// create the assign procs only for the primary region using the targetServer
TransitRegionStateProcedure[] primaryRegionProcs = regions.stream()
.map(env.getAssignmentManager().getRegionStates()::getOrCreateRegionStateNode)
TransitRegionStateProcedure[] primaryRegionProcs =
regions.stream().map(env.getAssignmentManager().getRegionStates()::getOrCreateRegionStateNode)
.map(regionNode -> {
TransitRegionStateProcedure proc =
TransitRegionStateProcedure.assign(env, regionNode.getRegionInfo(), targetServer);
TransitRegionStateProcedure.assign(env, regionNode.getRegionInfo(), targetServer);
regionNode.lock();
try {
// should never fail, as we have the exclusive region lock, and the region is newly
// created, or has been successfully closed so should not be on any servers, so SCP will
// not process it either.
assert !regionNode.isInTransition();
if (ignoreIfInTransition) {
if (regionNode.isInTransition()) {
return null;
}
} else {
// should never fail, as we have the exclusive region lock, and the region is newly
// created, or has been successfully closed so should not be on any servers, so SCP
// will
// not process it either.
assert !regionNode.isInTransition();
}
regionNode.setProcedure(proc);
} finally {
regionNode.unlock();
}
return proc;
}).toArray(TransitRegionStateProcedure[]::new);
}).filter(p -> p != null).toArray(TransitRegionStateProcedure[]::new);
if (regionReplication == DEFAULT_REGION_REPLICA) {
// this is the default case
return primaryRegionProcs;
@ -184,14 +203,16 @@ final class AssignmentManagerUtil {
static TransitRegionStateProcedure[] createAssignProceduresForOpeningNewRegions(
MasterProcedureEnv env, List<RegionInfo> regions, int regionReplication,
ServerName targetServer) {
return createAssignProcedures(env, regions, regionReplication, targetServer);
return createAssignProcedures(env, regions, regionReplication, targetServer, false);
}
static void reopenRegionsForRollback(MasterProcedureEnv env, List<RegionInfo> regions,
int regionReplication, ServerName targetServer) {
TransitRegionStateProcedure[] procs =
createAssignProcedures(env, regions, regionReplication, targetServer);
env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
createAssignProcedures(env, regions, regionReplication, targetServer, true);
if (procs.length > 0) {
env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
}
}
static void removeNonDefaultReplicas(MasterProcedureEnv env, Stream<RegionInfo> regions,

View File

@ -24,7 +24,6 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -66,8 +66,6 @@ public class TestMergeTableRegionsProcedure {
public final TestName name = new TestName();
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static final int initialRegionCount = 4;
private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
@ -108,9 +106,8 @@ public class TestMergeTableRegionsProcedure {
@Before
public void setup() throws Exception {
resetProcExecutorTestingKillFlag();
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
// Turn off balancer so it doesn't cut in and mess up our placements.
admin.balancerSwitch(false, true);
// Turn off the meta scanner so it don't remove parent on us.
@ -265,12 +262,15 @@ public class TestMergeTableRegionsProcedure {
long procId = procExec.submitProcedure(
new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
// Failing before MERGE_TABLE_REGIONS_CREATE_MERGED_REGION we should trigger the rollback
// NOTE: the 5 (number before MERGE_TABLE_REGIONS_CREATE_MERGED_REGION step) is
// Failing before MERGE_TABLE_REGIONS_UPDATE_META we should trigger the rollback
// NOTE: the 8 (number of MERGE_TABLE_REGIONS_UPDATE_META step) is
// hardcoded, so you have to look at this test at least once when you add a new step.
int numberOfSteps = 5;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
true);
int lastStep = 8;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep, true);
assertEquals(initialRegionCount, UTIL.getAdmin().getRegions(tableName).size());
UTIL.waitUntilAllRegionsAssigned(tableName);
List<HRegion> regions = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertEquals(initialRegionCount, regions.size());
}
@Test

View File

@ -348,7 +348,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
@ -366,18 +366,19 @@ public class TestSplitTableRegionProcedure {
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
// Failing before SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS we should trigger the
// Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
// rollback
// NOTE: the 3 (number before SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS step) is
// NOTE: the 7 (number of SPLIT_TABLE_REGION_UPDATE_META step) is
// hardcoded, so you have to look at this test at least once when you add a new step.
int numberOfSteps = 3;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
int lastStep = 7;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep,
true);
// check that we have only 1 region
assertEquals(1, UTIL.getAdmin().getRegions(tableName).size());
List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertEquals(1, daughters.size());
verifyData(daughters.get(0), startRowNum, rowCount,
UTIL.waitUntilAllRegionsAssigned(tableName);
List<HRegion> newRegions = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertEquals(1, newRegions.size());
verifyData(newRegions.get(0), startRowNum, rowCount,
Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes(ColumnFamilyName2));
assertEquals(splitSubmittedCount + 1, splitProcMetrics.getSubmittedCounter().getCount());

View File

@ -379,7 +379,7 @@ public class MasterProcedureTestingUtility {
*/
public static void testRecoveryAndDoubleExecution(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int numSteps, final boolean expectExecRunning) throws Exception {
final int lastStep, final boolean expectExecRunning) throws Exception {
ProcedureTestingUtility.waitProcedure(procExec, procId);
assertEquals(false, procExec.isRunning());
@ -397,10 +397,13 @@ public class MasterProcedureTestingUtility {
// fix would be get all visited states by the procedure and then check if user speccified
// state is in that list. Current assumption of sequential proregression of steps/ states is
// made at multiple places so we can keep while condition below for simplicity.
Procedure proc = procExec.getProcedure(procId);
Procedure<?> proc = procExec.getProcedure(procId);
int stepNum = proc instanceof StateMachineProcedure ?
((StateMachineProcedure) proc).getCurrentStateId() : 0;
while (stepNum < numSteps) {
for (;;) {
if (stepNum == lastStep) {
break;
}
LOG.info("Restart " + stepNum + " exec state=" + proc);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
restartMasterProcedureExecutor(procExec);

View File

@ -177,8 +177,8 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
long procId = procExec.submitProcedure(
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 2; // failing before CLONE_SNAPSHOT_WRITE_FS_LAYOUT
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), clonedTableName);

View File

@ -215,8 +215,8 @@ public class TestCreateNamespaceProcedure {
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 2; // failing before CREATE_NAMESPACE_CREATE_DIRECTORY
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
// Validate the non-existence of namespace
try {

View File

@ -186,8 +186,8 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 2; // failing before CREATE_TABLE_WRITE_FS_LAYOUT
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
TableName tableName = htd.getTableName();
MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
@ -239,7 +239,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
}
}
@Test(timeout = 60000)
@Test
public void testOnHDFSFailure() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());

View File

@ -25,10 +25,10 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -88,7 +88,7 @@ public class TestDeleteNamespaceProcedure {
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
for (TableDescriptor htd: UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
@ -200,8 +200,8 @@ public class TestDeleteNamespaceProcedure {
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 2; // failing before DELETE_NAMESPACE_DELETE_FROM_NS_TABLE
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
// Validate the namespace still exists
NamespaceDescriptor createdNsDescriptor=

View File

@ -138,8 +138,8 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 3; // fail before ENABLE_TABLE_SET_ENABLING_TABLE_STATE
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
}
}

View File

@ -24,9 +24,9 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -81,7 +81,7 @@ public class TestModifyNamespaceProcedure {
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
for (TableDescriptor htd: UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
@ -246,8 +246,8 @@ public class TestModifyNamespaceProcedure {
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 2; // failing before MODIFY_NAMESPACE_UPDATE_NS_TABLE
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
// Validate
NamespaceDescriptor currentNsDescriptor =

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -288,11 +287,10 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Modify multiple properties of the table.
final HTableDescriptor htd =
new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
htd.setCompactionEnabled(newCompactionEnableOption);
htd.addFamily(new HColumnDescriptor(cf2));
TableDescriptor td = UTIL.getAdmin().getDescriptor(tableName);
TableDescriptor newTd =
TableDescriptorBuilder.newBuilder(td).setCompactionEnabled(!td.isCompactionEnabled())
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf2)).build();
PerClientRandomNonceGenerator nonceGenerator = PerClientRandomNonceGenerator.get();
long nonceGroup = nonceGenerator.getNonceGroup();
@ -302,7 +300,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
// Start the Modify procedure && kill the executor
final long procId = procExec
.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceKey);
.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), newTd), nonceKey);
// Restart the executor after MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR and try to add column family
// as nonce are there , we should not fail
@ -328,11 +326,11 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
}
// Validate descriptor
HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
assertEquals(3, currentHtd.getFamiliesKeys().size());
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
assertEquals(!td.isCompactionEnabled(), currentHtd.isCompactionEnabled());
assertEquals(3, currentHtd.getColumnFamilyCount());
assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
// cf2 should be added
MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
@ -351,17 +349,17 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
htd.setCompactionEnabled(newCompactionEnableOption);
htd.addFamily(new HColumnDescriptor(familyName));
TableDescriptor td = UTIL.getAdmin().getDescriptor(tableName);
TableDescriptor newTd =
TableDescriptorBuilder.newBuilder(td).setCompactionEnabled(!td.isCompactionEnabled())
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(familyName)).build();
// Start the Modify procedure && kill the executor
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd));
long procId =
procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), newTd));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 3; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
// cf2 should not be present
MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
@ -382,19 +380,19 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
htd.setCompactionEnabled(newCompactionEnableOption);
htd.addFamily(new HColumnDescriptor(familyName));
htd.setRegionReplication(3);
TableDescriptor td = UTIL.getAdmin().getDescriptor(tableName);
TableDescriptor newTd =
TableDescriptorBuilder.newBuilder(td).setCompactionEnabled(!td.isCompactionEnabled())
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(familyName)).setRegionReplication(3)
.build();
// Start the Modify procedure && kill the executor
long procId = procExec.submitProcedure(
new ModifyTableProcedure(procExec.getEnvironment(), htd));
new ModifyTableProcedure(procExec.getEnvironment(), newTd));
// Restart the executor and rollback the step twice
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
int lastStep = 3; // failing before MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep);
// cf2 should not be present
MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),