HBASE-17149 Procedure v2 - Fix nonce submission
Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
parent
a5ee36d937
commit
da356069f2
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.NonceKey;
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Thread Pool that executes the submitted procedures.
|
* Thread Pool that executes the submitted procedures.
|
||||||
|
@ -653,47 +654,136 @@ public class ProcedureExecutor<TEnvironment> {
|
||||||
return timeoutExecutor.remove(chore);
|
return timeoutExecutor.remove(chore);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ==========================================================================
|
||||||
|
// Nonce Procedure helpers
|
||||||
|
// ==========================================================================
|
||||||
|
/**
|
||||||
|
* Create a NoneKey from the specified nonceGroup and nonce.
|
||||||
|
* @param nonceGroup
|
||||||
|
* @param nonce
|
||||||
|
* @return the generated NonceKey
|
||||||
|
*/
|
||||||
|
public NonceKey createNonceKey(final long nonceGroup, final long nonce) {
|
||||||
|
return (nonce == HConstants.NO_NONCE) ? null : new NonceKey(nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Register a nonce for a procedure that is going to be submitted.
|
||||||
|
* A procId will be reserved and on submitProcedure(),
|
||||||
|
* the procedure with the specified nonce will take the reserved ProcId.
|
||||||
|
* If someone already reserved the nonce, this method will return the procId reserved,
|
||||||
|
* otherwise an invalid procId will be returned. and the caller should procede
|
||||||
|
* and submit the procedure.
|
||||||
|
*
|
||||||
|
* @param nonceKey A unique identifier for this operation from the client or process.
|
||||||
|
* @return the procId associated with the nonce, if any otherwise an invalid procId.
|
||||||
|
*/
|
||||||
|
public long registerNonce(final NonceKey nonceKey) {
|
||||||
|
if (nonceKey == null) return -1;
|
||||||
|
|
||||||
|
// check if we have already a Reserved ID for the nonce
|
||||||
|
Long oldProcId = nonceKeysToProcIdsMap.get(nonceKey);
|
||||||
|
if (oldProcId == null) {
|
||||||
|
// reserve a new Procedure ID, this will be associated with the nonce
|
||||||
|
// and the procedure submitted with the specified nonce will use this ID.
|
||||||
|
final long newProcId = nextProcId();
|
||||||
|
oldProcId = nonceKeysToProcIdsMap.putIfAbsent(nonceKey, newProcId);
|
||||||
|
if (oldProcId == null) return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// we found a registered nonce, but the procedure may not have been submitted yet.
|
||||||
|
// since the client expect the procedure to be submitted, spin here until it is.
|
||||||
|
final boolean isTraceEnabled = LOG.isTraceEnabled();
|
||||||
|
while (isRunning() &&
|
||||||
|
!(procedures.containsKey(oldProcId) || completed.containsKey(oldProcId)) &&
|
||||||
|
nonceKeysToProcIdsMap.containsKey(nonceKey)) {
|
||||||
|
if (isTraceEnabled) {
|
||||||
|
LOG.trace("waiting for procId=" + oldProcId.longValue() + " to be submitted");
|
||||||
|
}
|
||||||
|
Threads.sleep(100);
|
||||||
|
}
|
||||||
|
return oldProcId.longValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove the NonceKey if the procedure was not submitted to the executor.
|
||||||
|
* @param nonceKey A unique identifier for this operation from the client or process.
|
||||||
|
*/
|
||||||
|
public void unregisterNonceIfProcedureWasNotSubmitted(final NonceKey nonceKey) {
|
||||||
|
if (nonceKey == null) return;
|
||||||
|
|
||||||
|
final Long procId = nonceKeysToProcIdsMap.get(nonceKey);
|
||||||
|
if (procId == null) return;
|
||||||
|
|
||||||
|
// if the procedure was not submitted, remove the nonce
|
||||||
|
if (!(procedures.containsKey(procId) || completed.containsKey(procId))) {
|
||||||
|
nonceKeysToProcIdsMap.remove(nonceKey);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If the failure failed before submitting it, we may want to give back the
|
||||||
|
* same error to the requests with the same nonceKey.
|
||||||
|
*
|
||||||
|
* @param nonceKey A unique identifier for this operation from the client or process
|
||||||
|
* @param procName name of the procedure, used to inform the user
|
||||||
|
* @param procOwner name of the owner of the procedure, used to inform the user
|
||||||
|
* @param exception the failure to report to the user
|
||||||
|
*/
|
||||||
|
public void setFailureResultForNonce(final NonceKey nonceKey, final String procName,
|
||||||
|
final User procOwner, final IOException exception) {
|
||||||
|
if (nonceKey == null) return;
|
||||||
|
|
||||||
|
final Long procId = nonceKeysToProcIdsMap.get(nonceKey);
|
||||||
|
if (procId == null || completed.containsKey(procId)) return;
|
||||||
|
|
||||||
|
final long currentTime = EnvironmentEdgeManager.currentTime();
|
||||||
|
final ProcedureInfo result = new ProcedureInfo(procId.longValue(),
|
||||||
|
procName, procOwner != null ? procOwner.getShortName() : null,
|
||||||
|
ProcedureUtil.convertToProcedureState(ProcedureState.ROLLEDBACK),
|
||||||
|
-1, nonceKey, exception, currentTime, currentTime, null);
|
||||||
|
completed.putIfAbsent(procId, result);
|
||||||
|
}
|
||||||
|
|
||||||
// ==========================================================================
|
// ==========================================================================
|
||||||
// Submit/Abort Procedure
|
// Submit/Abort Procedure
|
||||||
// ==========================================================================
|
// ==========================================================================
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a new root-procedure to the executor.
|
* Add a new root-procedure to the executor.
|
||||||
* @param proc the new procedure to execute.
|
* @param proc the new procedure to execute.
|
||||||
* @return the procedure id, that can be used to monitor the operation
|
* @return the procedure id, that can be used to monitor the operation
|
||||||
*/
|
*/
|
||||||
public long submitProcedure(final Procedure proc) {
|
public long submitProcedure(final Procedure proc) {
|
||||||
return submitProcedure(proc, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return submitProcedure(proc, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a new root-procedure to the executor.
|
* Add a new root-procedure to the executor.
|
||||||
* @param proc the new procedure to execute.
|
* @param proc the new procedure to execute.
|
||||||
* @param nonceGroup
|
* @param nonceKey the registered unique identifier for this operation from the client or process.
|
||||||
* @param nonce
|
|
||||||
* @return the procedure id, that can be used to monitor the operation
|
* @return the procedure id, that can be used to monitor the operation
|
||||||
*/
|
*/
|
||||||
public long submitProcedure(final Procedure proc, final long nonceGroup, final long nonce) {
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
|
||||||
|
justification = "FindBugs is blind to the check-for-null")
|
||||||
|
public long submitProcedure(final Procedure proc, final NonceKey nonceKey) {
|
||||||
Preconditions.checkArgument(lastProcId.get() >= 0);
|
Preconditions.checkArgument(lastProcId.get() >= 0);
|
||||||
Preconditions.checkArgument(isRunning(), "executor not running");
|
Preconditions.checkArgument(isRunning(), "executor not running");
|
||||||
|
|
||||||
// Prepare procedure
|
|
||||||
prepareProcedure(proc);
|
prepareProcedure(proc);
|
||||||
|
|
||||||
// Check whether the proc exists. If exist, just return the proc id.
|
final Long currentProcId;
|
||||||
// This is to prevent the same proc to submit multiple times (it could happen
|
if (nonceKey != null) {
|
||||||
// when client could not talk to server and resubmit the same request).
|
currentProcId = nonceKeysToProcIdsMap.get(nonceKey);
|
||||||
if (nonce != HConstants.NO_NONCE) {
|
Preconditions.checkArgument(currentProcId != null,
|
||||||
final NonceKey noncekey = new NonceKey(nonceGroup, nonce);
|
"expected nonceKey=" + nonceKey + " to be reserved, use registerNonce()");
|
||||||
proc.setNonceKey(noncekey);
|
} else {
|
||||||
|
currentProcId = nextProcId();
|
||||||
Long oldProcId = nonceKeysToProcIdsMap.putIfAbsent(noncekey, proc.getProcId());
|
|
||||||
if (oldProcId != null) {
|
|
||||||
// Found the proc
|
|
||||||
return oldProcId.longValue();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Initialize the procedure
|
||||||
|
proc.setNonceKey(nonceKey);
|
||||||
|
proc.setProcId(currentProcId.longValue());
|
||||||
|
|
||||||
// Commit the transaction
|
// Commit the transaction
|
||||||
store.insert(proc, null);
|
store.insert(proc, null);
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -708,13 +798,14 @@ public class ProcedureExecutor<TEnvironment> {
|
||||||
* Add a set of new root-procedure to the executor.
|
* Add a set of new root-procedure to the executor.
|
||||||
* @param procs the new procedures to execute.
|
* @param procs the new procedures to execute.
|
||||||
*/
|
*/
|
||||||
|
// TODO: Do we need to take nonces here?
|
||||||
public void submitProcedures(final Procedure[] procs) {
|
public void submitProcedures(final Procedure[] procs) {
|
||||||
Preconditions.checkArgument(lastProcId.get() >= 0);
|
Preconditions.checkArgument(lastProcId.get() >= 0);
|
||||||
Preconditions.checkArgument(isRunning(), "executor not running");
|
Preconditions.checkArgument(isRunning(), "executor not running");
|
||||||
|
|
||||||
// Prepare procedure
|
// Prepare procedure
|
||||||
for (int i = 0; i < procs.length; ++i) {
|
for (int i = 0; i < procs.length; ++i) {
|
||||||
prepareProcedure(procs[i]);
|
prepareProcedure(procs[i]).setProcId(nextProcId());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Commit the transaction
|
// Commit the transaction
|
||||||
|
@ -729,17 +820,14 @@ public class ProcedureExecutor<TEnvironment> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void prepareProcedure(final Procedure proc) {
|
private Procedure prepareProcedure(final Procedure proc) {
|
||||||
Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
|
Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
|
||||||
Preconditions.checkArgument(isRunning(), "executor not running");
|
Preconditions.checkArgument(isRunning(), "executor not running");
|
||||||
Preconditions.checkArgument(!proc.hasParent(), "unexpected parent", proc);
|
Preconditions.checkArgument(!proc.hasParent(), "unexpected parent", proc);
|
||||||
if (this.checkOwnerSet) {
|
if (this.checkOwnerSet) {
|
||||||
Preconditions.checkArgument(proc.hasOwner(), "missing owner");
|
Preconditions.checkArgument(proc.hasOwner(), "missing owner");
|
||||||
}
|
}
|
||||||
|
return proc;
|
||||||
// Initialize the Procedure ID
|
|
||||||
final long currentProcId = nextProcId();
|
|
||||||
proc.setProcId(currentProcId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private long pushProcedure(final Procedure proc) {
|
private long pushProcedure(final Procedure proc) {
|
||||||
|
@ -754,7 +842,7 @@ public class ProcedureExecutor<TEnvironment> {
|
||||||
procedures.put(currentProcId, proc);
|
procedures.put(currentProcId, proc);
|
||||||
sendProcedureAddedNotification(currentProcId);
|
sendProcedureAddedNotification(currentProcId);
|
||||||
scheduler.addBack(proc);
|
scheduler.addBack(proc);
|
||||||
return currentProcId;
|
return proc.getProcId();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator
|
||||||
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
|
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
|
||||||
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
@ -178,13 +179,20 @@ public class ProcedureTestingUtility {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc,
|
public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc,
|
||||||
final long nonceGroup,
|
final long nonceGroup, final long nonce) {
|
||||||
final long nonce) {
|
long procId = submitProcedure(procExecutor, proc, nonceGroup, nonce);
|
||||||
long procId = procExecutor.submitProcedure(proc, nonceGroup, nonce);
|
|
||||||
waitProcedure(procExecutor, procId);
|
waitProcedure(procExecutor, procId);
|
||||||
return procId;
|
return procId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static <TEnv> long submitProcedure(ProcedureExecutor<TEnv> procExecutor, Procedure proc,
|
||||||
|
final long nonceGroup, final long nonce) {
|
||||||
|
final NonceKey nonceKey = procExecutor.createNonceKey(nonceGroup, nonce);
|
||||||
|
long procId = procExecutor.registerNonce(nonceKey);
|
||||||
|
assertFalse(procId >= 0);
|
||||||
|
return procExecutor.submitProcedure(proc, nonceKey);
|
||||||
|
}
|
||||||
|
|
||||||
public static <TEnv> void waitProcedure(ProcedureExecutor<TEnv> procExecutor, Procedure proc) {
|
public static <TEnv> void waitProcedure(ProcedureExecutor<TEnv> procExecutor, Procedure proc) {
|
||||||
while (proc.getState() == ProcedureState.INITIALIZING) {
|
while (proc.getState() == ProcedureState.INITIALIZING) {
|
||||||
Threads.sleepWithoutInterrupt(250);
|
Threads.sleepWithoutInterrupt(250);
|
||||||
|
|
|
@ -0,0 +1,285 @@
|
||||||
|
/**
|
||||||
|
* 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.procedure2;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
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, SmallTests.class})
|
||||||
|
public class TestProcedureNonce {
|
||||||
|
private static final Log LOG = LogFactory.getLog(TestProcedureNonce.class);
|
||||||
|
|
||||||
|
private static final int PROCEDURE_EXECUTOR_SLOTS = 2;
|
||||||
|
|
||||||
|
private static TestProcEnv procEnv;
|
||||||
|
private static ProcedureExecutor<TestProcEnv> procExecutor;
|
||||||
|
private static ProcedureStore procStore;
|
||||||
|
|
||||||
|
private HBaseCommonTestingUtility htu;
|
||||||
|
private FileSystem fs;
|
||||||
|
private Path logDir;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws IOException {
|
||||||
|
htu = new HBaseCommonTestingUtility();
|
||||||
|
Path testDir = htu.getDataTestDir();
|
||||||
|
fs = testDir.getFileSystem(htu.getConfiguration());
|
||||||
|
assertTrue(testDir.depth() > 1);
|
||||||
|
|
||||||
|
logDir = new Path(testDir, "proc-logs");
|
||||||
|
procEnv = new TestProcEnv();
|
||||||
|
procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), fs, logDir);
|
||||||
|
procExecutor = new ProcedureExecutor(htu.getConfiguration(), procEnv, procStore);
|
||||||
|
procExecutor.testing = new ProcedureExecutor.Testing();
|
||||||
|
procStore.start(PROCEDURE_EXECUTOR_SLOTS);
|
||||||
|
procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws IOException {
|
||||||
|
procExecutor.stop();
|
||||||
|
procStore.stop(false);
|
||||||
|
fs.delete(logDir, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=30000)
|
||||||
|
public void testCompletedProcWithSameNonce() throws Exception {
|
||||||
|
final long nonceGroup = 123;
|
||||||
|
final long nonce = 2222;
|
||||||
|
|
||||||
|
// register the nonce
|
||||||
|
final NonceKey nonceKey = procExecutor.createNonceKey(nonceGroup, nonce);
|
||||||
|
assertFalse(procExecutor.registerNonce(nonceKey) >= 0);
|
||||||
|
|
||||||
|
// Submit a proc and wait for its completion
|
||||||
|
Procedure proc = new TestSingleStepProcedure();
|
||||||
|
long procId = procExecutor.submitProcedure(proc, nonceKey);
|
||||||
|
ProcedureTestingUtility.waitProcedure(procExecutor, procId);
|
||||||
|
|
||||||
|
// Restart
|
||||||
|
ProcedureTestingUtility.restart(procExecutor);
|
||||||
|
ProcedureTestingUtility.waitProcedure(procExecutor, procId);
|
||||||
|
|
||||||
|
// try to register a procedure with the same nonce
|
||||||
|
// we should get back the old procId
|
||||||
|
assertEquals(procId, procExecutor.registerNonce(nonceKey));
|
||||||
|
|
||||||
|
ProcedureInfo result = procExecutor.getResult(procId);
|
||||||
|
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=30000)
|
||||||
|
public void testRunningProcWithSameNonce() throws Exception {
|
||||||
|
final long nonceGroup = 456;
|
||||||
|
final long nonce = 33333;
|
||||||
|
|
||||||
|
// register the nonce
|
||||||
|
final NonceKey nonceKey = procExecutor.createNonceKey(nonceGroup, nonce);
|
||||||
|
assertFalse(procExecutor.registerNonce(nonceKey) >= 0);
|
||||||
|
|
||||||
|
// Submit a proc and use a latch to prevent the step execution until we submitted proc2
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
TestSingleStepProcedure proc = new TestSingleStepProcedure();
|
||||||
|
procEnv.setWaitLatch(latch);
|
||||||
|
long procId = procExecutor.submitProcedure(proc, nonceKey);
|
||||||
|
while (proc.step != 1) Threads.sleep(25);
|
||||||
|
|
||||||
|
// try to register a procedure with the same nonce
|
||||||
|
// we should get back the old procId
|
||||||
|
assertEquals(procId, procExecutor.registerNonce(nonceKey));
|
||||||
|
|
||||||
|
// complete the procedure
|
||||||
|
latch.countDown();
|
||||||
|
|
||||||
|
// Restart, the procedure is not completed yet
|
||||||
|
ProcedureTestingUtility.restart(procExecutor);
|
||||||
|
ProcedureTestingUtility.waitProcedure(procExecutor, procId);
|
||||||
|
|
||||||
|
// try to register a procedure with the same nonce
|
||||||
|
// we should get back the old procId
|
||||||
|
assertEquals(procId, procExecutor.registerNonce(nonceKey));
|
||||||
|
|
||||||
|
ProcedureInfo result = procExecutor.getResult(procId);
|
||||||
|
ProcedureTestingUtility.assertProcNotFailed(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetFailureResultForNonce() throws IOException {
|
||||||
|
final long nonceGroup = 234;
|
||||||
|
final long nonce = 55555;
|
||||||
|
|
||||||
|
// check and register the request nonce
|
||||||
|
final NonceKey nonceKey = procExecutor.createNonceKey(nonceGroup, nonce);
|
||||||
|
assertFalse(procExecutor.registerNonce(nonceKey) >= 0);
|
||||||
|
|
||||||
|
procExecutor.setFailureResultForNonce(nonceKey, "testProc", User.getCurrent(),
|
||||||
|
new IOException("test failure"));
|
||||||
|
|
||||||
|
final long procId = procExecutor.registerNonce(nonceKey);
|
||||||
|
ProcedureInfo result = procExecutor.getResult(procId);
|
||||||
|
ProcedureTestingUtility.assertProcFailed(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=30000)
|
||||||
|
public void testConcurrentNonceRegistration() throws IOException {
|
||||||
|
testConcurrentNonceRegistration(true, 567, 44444);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=30000)
|
||||||
|
public void testConcurrentNonceRegistrationWithRollback() throws IOException {
|
||||||
|
testConcurrentNonceRegistration(false, 890, 55555);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testConcurrentNonceRegistration(final boolean submitProcedure,
|
||||||
|
final long nonceGroup, final long nonce) throws IOException {
|
||||||
|
// register the nonce
|
||||||
|
final NonceKey nonceKey = procExecutor.createNonceKey(nonceGroup, nonce);
|
||||||
|
|
||||||
|
final AtomicReference<Throwable> t1Exception = new AtomicReference();
|
||||||
|
final AtomicReference<Throwable> t2Exception = new AtomicReference();
|
||||||
|
|
||||||
|
final CountDownLatch t1NonceRegisteredLatch = new CountDownLatch(1);
|
||||||
|
final CountDownLatch t2BeforeNonceRegisteredLatch = new CountDownLatch(1);
|
||||||
|
final Thread[] threads = new Thread[2];
|
||||||
|
threads[0] = new Thread() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
// release the nonce and wake t2
|
||||||
|
assertFalse("unexpected already registered nonce",
|
||||||
|
procExecutor.registerNonce(nonceKey) >= 0);
|
||||||
|
t1NonceRegisteredLatch.countDown();
|
||||||
|
|
||||||
|
// hold the submission until t2 is registering the nonce
|
||||||
|
t2BeforeNonceRegisteredLatch.await();
|
||||||
|
Threads.sleep(1000);
|
||||||
|
|
||||||
|
if (submitProcedure) {
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
TestSingleStepProcedure proc = new TestSingleStepProcedure();
|
||||||
|
procEnv.setWaitLatch(latch);
|
||||||
|
|
||||||
|
procExecutor.submitProcedure(proc, nonceKey);
|
||||||
|
Threads.sleep(100);
|
||||||
|
|
||||||
|
// complete the procedure
|
||||||
|
latch.countDown();
|
||||||
|
} else {
|
||||||
|
procExecutor.unregisterNonceIfProcedureWasNotSubmitted(nonceKey);
|
||||||
|
}
|
||||||
|
} catch (Throwable e) {
|
||||||
|
t1Exception.set(e);
|
||||||
|
} finally {
|
||||||
|
t1NonceRegisteredLatch.countDown();
|
||||||
|
t2BeforeNonceRegisteredLatch.countDown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
threads[1] = new Thread() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
// wait until t1 has registered the nonce
|
||||||
|
t1NonceRegisteredLatch.await();
|
||||||
|
|
||||||
|
// register the nonce
|
||||||
|
t2BeforeNonceRegisteredLatch.countDown();
|
||||||
|
assertFalse("unexpected non registered nonce",
|
||||||
|
procExecutor.registerNonce(nonceKey) < 0);
|
||||||
|
} catch (Throwable e) {
|
||||||
|
t2Exception.set(e);
|
||||||
|
} finally {
|
||||||
|
t1NonceRegisteredLatch.countDown();
|
||||||
|
t2BeforeNonceRegisteredLatch.countDown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
for (int i = 0; i < threads.length; ++i) threads[i].start();
|
||||||
|
for (int i = 0; i < threads.length; ++i) Threads.shutdown(threads[i]);
|
||||||
|
ProcedureTestingUtility.waitNoProcedureRunning(procExecutor);
|
||||||
|
assertEquals(null, t1Exception.get());
|
||||||
|
assertEquals(null, t2Exception.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class TestSingleStepProcedure extends SequentialProcedure<TestProcEnv> {
|
||||||
|
private int step = 0;
|
||||||
|
|
||||||
|
public TestSingleStepProcedure() { }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Procedure[] execute(TestProcEnv env) throws InterruptedException {
|
||||||
|
step++;
|
||||||
|
env.waitOnLatch();
|
||||||
|
LOG.debug("execute procedure " + this + " step=" + step);
|
||||||
|
step++;
|
||||||
|
setResult(Bytes.toBytes(step));
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void rollback(TestProcEnv env) { }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean abort(TestProcEnv env) { return true; }
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class TestProcEnv {
|
||||||
|
private CountDownLatch latch = null;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* set/unset a latch. every procedure execute() step will wait on the latch if any.
|
||||||
|
*/
|
||||||
|
public void setWaitLatch(CountDownLatch latch) {
|
||||||
|
this.latch = latch;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void waitOnLatch() throws InterruptedException {
|
||||||
|
if (latch != null) {
|
||||||
|
latch.await();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -288,49 +288,6 @@ public class TestProcedureRecovery {
|
||||||
ProcedureTestingUtility.assertIsAbortException(result);
|
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();
|
|
||||||
waitProcedure(procId);
|
|
||||||
|
|
||||||
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);
|
|
||||||
|
|
||||||
ProcedureInfo 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 TestSingleStepProcedure();
|
|
||||||
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc, nonceGroup, nonce);
|
|
||||||
|
|
||||||
// Restart (use a latch to prevent the step execution until we submitted proc2)
|
|
||||||
CountDownLatch latch = new CountDownLatch(1);
|
|
||||||
procEnv.setWaitLatch(latch);
|
|
||||||
restart();
|
|
||||||
// Submit a procedure with the same nonce and expect the same procedure would return.
|
|
||||||
Procedure proc2 = new TestSingleStepProcedure();
|
|
||||||
long procId2 = procExecutor.submitProcedure(proc2, nonceGroup, nonce);
|
|
||||||
latch.countDown();
|
|
||||||
procEnv.setWaitLatch(null);
|
|
||||||
|
|
||||||
// The original proc is not completed and the new submission should have the same proc Id.
|
|
||||||
assertTrue(procId == procId2);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public static class TestStateMachineProcedure
|
public static class TestStateMachineProcedure
|
||||||
extends StateMachineProcedure<TestProcEnv, TestStateMachineProcedure.State> {
|
extends StateMachineProcedure<TestProcEnv, TestStateMachineProcedure.State> {
|
||||||
enum State { STATE_1, STATE_2, STATE_3, DONE }
|
enum State { STATE_1, STATE_2, STATE_3, DONE }
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* View and edit the current cluster schema. Use this API making any modification to
|
* View and edit the current cluster schema. Use this API making any modification to
|
||||||
|
@ -77,39 +78,33 @@ public interface ClusterSchema {
|
||||||
/**
|
/**
|
||||||
* Create a new Namespace.
|
* Create a new Namespace.
|
||||||
* @param namespaceDescriptor descriptor for new Namespace
|
* @param namespaceDescriptor descriptor for new Namespace
|
||||||
* @param nonceGroup Identifier for the source of the request, a client or process.
|
* @param nonceKey A unique identifier for this operation from the client or process.
|
||||||
* @param nonce A unique identifier for this operation from the client or process identified by
|
|
||||||
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
* as well as {@link IOException}
|
* as well as {@link IOException}
|
||||||
*/
|
*/
|
||||||
long createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup, long nonce)
|
long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Modify an existing Namespace.
|
* Modify an existing Namespace.
|
||||||
* @param nonceGroup Identifier for the source of the request, a client or process.
|
* @param nonceKey A unique identifier for this operation from the client or process.
|
||||||
* @param nonce A unique identifier for this operation from the client or process identified by
|
|
||||||
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
* as well as {@link IOException}
|
* as well as {@link IOException}
|
||||||
*/
|
*/
|
||||||
long modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce)
|
long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete an existing Namespace.
|
* Delete an existing Namespace.
|
||||||
* Only empty Namespaces (no tables) can be removed.
|
* Only empty Namespaces (no tables) can be removed.
|
||||||
* @param nonceGroup Identifier for the source of the request, a client or process.
|
* @param nonceKey A unique identifier for this operation from the client or process.
|
||||||
* @param nonce A unique identifier for this operation from the client or process identified by
|
|
||||||
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
* as well as {@link IOException}
|
* as well as {@link IOException}
|
||||||
*/
|
*/
|
||||||
long deleteNamespace(String name, long nonceGroup, long nonce)
|
long deleteNamespace(String name, NonceKey nonceKey)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
|
import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
|
||||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
class ClusterSchemaServiceImpl implements ClusterSchemaService {
|
class ClusterSchemaServiceImpl implements ClusterSchemaService {
|
||||||
|
@ -78,38 +79,35 @@ class ClusterSchemaServiceImpl implements ClusterSchemaService {
|
||||||
return this.tableNamespaceManager;
|
return this.tableNamespaceManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
private long submitProcedure(final Procedure<?> procedure, long nonceGroup,
|
private long submitProcedure(final Procedure<?> procedure, final NonceKey nonceKey)
|
||||||
long nonce)
|
throws ServiceNotRunningException {
|
||||||
throws ServiceNotRunningException {
|
|
||||||
checkIsRunning();
|
checkIsRunning();
|
||||||
ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
|
ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
|
||||||
return pe.submitProcedure(procedure, nonceGroup, nonce);
|
return pe.submitProcedure(procedure, nonceKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long createNamespace(NamespaceDescriptor namespaceDescriptor,
|
public long createNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey)
|
||||||
long nonceGroup, long nonce)
|
throws IOException {
|
||||||
throws IOException {
|
|
||||||
return submitProcedure(new CreateNamespaceProcedure(
|
return submitProcedure(new CreateNamespaceProcedure(
|
||||||
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
||||||
nonceGroup, nonce);
|
nonceKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long modifyNamespace(NamespaceDescriptor namespaceDescriptor,
|
public long modifyNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey)
|
||||||
long nonceGroup, long nonce)
|
throws IOException {
|
||||||
throws IOException {
|
|
||||||
return submitProcedure(new ModifyNamespaceProcedure(
|
return submitProcedure(new ModifyNamespaceProcedure(
|
||||||
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
||||||
nonceGroup, nonce);
|
nonceKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long deleteNamespace(String name, long nonceGroup, long nonce)
|
public long deleteNamespace(String name, final NonceKey nonceKey)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return submitProcedure(new DeleteNamespaceProcedure(
|
return submitProcedure(new DeleteNamespaceProcedure(
|
||||||
this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
|
||||||
nonceGroup, nonce);
|
nonceKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -114,6 +114,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
|
import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
|
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
|
||||||
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
||||||
import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
|
import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
|
||||||
|
@ -145,16 +146,17 @@ import org.apache.hadoop.hbase.security.UserProvider;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
|
||||||
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
|
||||||
import org.apache.hadoop.hbase.util.Addressing;
|
import org.apache.hadoop.hbase.util.Addressing;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.CompressionTest;
|
import org.apache.hadoop.hbase.util.CompressionTest;
|
||||||
import org.apache.hadoop.hbase.util.EncryptionTest;
|
import org.apache.hadoop.hbase.util.EncryptionTest;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||||
import org.apache.hadoop.hbase.util.HasThread;
|
import org.apache.hadoop.hbase.util.HasThread;
|
||||||
import org.apache.hadoop.hbase.util.IdLock;
|
import org.apache.hadoop.hbase.util.IdLock;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.hbase.util.VersionInfo;
|
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||||
|
@ -1433,23 +1435,26 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
regionsToMerge [0] = regionInfoA;
|
regionsToMerge [0] = regionInfoA;
|
||||||
regionsToMerge [1] = regionInfoB;
|
regionsToMerge [1] = regionInfoB;
|
||||||
|
|
||||||
if (cpHost != null) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
cpHost.preDispatchMerge(regionInfoA, regionInfoB);
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
}
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
getMaster().getMasterCoprocessorHost().preDispatchMerge(regionInfoA, regionInfoB);
|
||||||
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " Merge regions "
|
LOG.info(getClientIdAuditPrefix() + " Merge regions "
|
||||||
+ regionInfoA.getEncodedName() + " and " + regionInfoB.getEncodedName());
|
+ regionInfoA.getEncodedName() + " and " + regionInfoB.getEncodedName());
|
||||||
|
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
submitProcedure(new DispatchMergingRegionsProcedure(procedureExecutor.getEnvironment(),
|
||||||
new DispatchMergingRegionsProcedure(
|
tableName, regionsToMerge, forcible));
|
||||||
procedureExecutor.getEnvironment(), tableName, regionsToMerge, forcible),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
getMaster().getMasterCoprocessorHost().postDispatchMerge(regionInfoA, regionInfoB);
|
||||||
cpHost.postDispatchMerge(regionInfoA, regionInfoB);
|
}
|
||||||
}
|
|
||||||
return procId;
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DisableTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1478,22 +1483,26 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
"Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
|
"Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (cpHost != null) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
cpHost.preMergeRegions(regionsToMerge);
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
}
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
|
||||||
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " Merge regions "
|
LOG.info(getClientIdAuditPrefix() + " Merge regions " +
|
||||||
+ regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName());
|
regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName());
|
||||||
|
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
|
||||||
new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(), regionsToMerge, forcible),
|
regionsToMerge, forcible));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
|
||||||
cpHost.postMergeRegions(regionsToMerge);
|
}
|
||||||
}
|
|
||||||
return procId;
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DisableTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1504,18 +1513,24 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
final long nonce) throws IOException {
|
final long nonce) throws IOException {
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
|
||||||
if (cpHost != null) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
cpHost.preSplitRegion(regionInfo.getTable(), splitRow);
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
}
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
|
||||||
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " Split region " + regionInfo);
|
LOG.info(getClientIdAuditPrefix() + " Split region " + regionInfo);
|
||||||
|
|
||||||
// Execute the operation asynchronously
|
// Execute the operation asynchronously
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
submitProcedure(new SplitTableRegionProcedure(procedureExecutor.getEnvironment(),
|
||||||
new SplitTableRegionProcedure(procedureExecutor.getEnvironment(), regionInfo, splitRow),
|
regionInfo, splitRow));
|
||||||
nonceGroup, nonce);
|
}
|
||||||
|
|
||||||
return procId;
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DisableTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
void move(final byte[] encodedRegionName,
|
void move(final byte[] encodedRegionName,
|
||||||
|
@ -1600,36 +1615,37 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
final byte [][] splitKeys,
|
final byte [][] splitKeys,
|
||||||
final long nonceGroup,
|
final long nonceGroup,
|
||||||
final long nonce) throws IOException {
|
final long nonce) throws IOException {
|
||||||
if (isStopped()) {
|
|
||||||
throw new MasterNotRunningException();
|
|
||||||
}
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
|
||||||
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
|
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
|
||||||
this.clusterSchemaService.getNamespace(namespace);
|
this.clusterSchemaService.getNamespace(namespace);
|
||||||
|
|
||||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
|
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
|
||||||
checkInitialized();
|
|
||||||
sanityCheckTableDescriptor(hTableDescriptor);
|
sanityCheckTableDescriptor(hTableDescriptor);
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preCreateTable(hTableDescriptor, newRegions);
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
|
|
||||||
|
|
||||||
// TODO: We can handle/merge duplicate requests, and differentiate the case of
|
return MasterProcedureUtil.submitProcedure(
|
||||||
// TableExistsException by saying if the schema is the same or not.
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
|
@Override
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
protected void run() throws IOException {
|
||||||
new CreateTableProcedure(
|
getMaster().getMasterCoprocessorHost().preCreateTable(hTableDescriptor, newRegions);
|
||||||
procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
|
||||||
cpHost.postCreateTable(hTableDescriptor, newRegions);
|
|
||||||
}
|
|
||||||
|
|
||||||
return procId;
|
// TODO: We can handle/merge duplicate requests, and differentiate the case of
|
||||||
|
// TableExistsException by saying if the schema is the same or not.
|
||||||
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
|
||||||
|
submitProcedure(new CreateTableProcedure(
|
||||||
|
procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postCreateTable(hTableDescriptor, newRegions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "CreateTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1968,24 +1984,29 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
final long nonceGroup,
|
final long nonceGroup,
|
||||||
final long nonce) throws IOException {
|
final long nonce) throws IOException {
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preDeleteTable(tableName);
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
|
|
||||||
|
|
||||||
// TODO: We can handle/merge duplicate request
|
return MasterProcedureUtil.submitProcedure(
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
@Override
|
||||||
new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch),
|
protected void run() throws IOException {
|
||||||
nonceGroup,
|
getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
|
||||||
cpHost.postDeleteTable(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
return procId;
|
// TODO: We can handle/merge duplicate request
|
||||||
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
|
||||||
|
submitProcedure(new DeleteTableProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DeleteTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1995,23 +2016,27 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
final long nonceGroup,
|
final long nonceGroup,
|
||||||
final long nonce) throws IOException {
|
final long nonce) throws IOException {
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preTruncateTable(tableName);
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
|
|
||||||
|
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
return MasterProcedureUtil.submitProcedure(
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName,
|
@Override
|
||||||
preserveSplits, latch),
|
protected void run() throws IOException {
|
||||||
nonceGroup,
|
getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
|
||||||
cpHost.postTruncateTable(tableName);
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||||
}
|
submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(),
|
||||||
return procId;
|
tableName, preserveSplits, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "TruncateTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2025,24 +2050,29 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
checkCompression(columnDescriptor);
|
checkCompression(columnDescriptor);
|
||||||
checkEncryption(conf, columnDescriptor);
|
checkEncryption(conf, columnDescriptor);
|
||||||
checkReplicationScope(columnDescriptor);
|
checkReplicationScope(columnDescriptor);
|
||||||
if (cpHost != null) {
|
|
||||||
if (cpHost.preAddColumn(tableName, columnDescriptor)) {
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
|
||||||
new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
|
|
||||||
columnDescriptor, latch),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
cpHost.postAddColumn(tableName, columnDescriptor);
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
}
|
@Override
|
||||||
return procId;
|
protected void run() throws IOException {
|
||||||
|
if (getMaster().getMasterCoprocessorHost().preAddColumn(tableName, columnDescriptor)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Execute the operation synchronously, wait for the operation to complete before continuing
|
||||||
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||||
|
submitProcedure(new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, columnDescriptor, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postAddColumn(tableName, columnDescriptor);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "AddColumnFamilyProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2056,26 +2086,31 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
checkCompression(descriptor);
|
checkCompression(descriptor);
|
||||||
checkEncryption(conf, descriptor);
|
checkEncryption(conf, descriptor);
|
||||||
checkReplicationScope(descriptor);
|
checkReplicationScope(descriptor);
|
||||||
if (cpHost != null) {
|
|
||||||
if (cpHost.preModifyColumn(tableName, descriptor)) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
return -1;
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
if (getMaster().getMasterCoprocessorHost().preModifyColumn(tableName, descriptor)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
|
||||||
|
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||||
|
submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, descriptor, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postModifyColumn(tableName, descriptor);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
|
|
||||||
|
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
@Override
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
protected String getDescription() {
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
return "ModifyColumnFamilyProcedure";
|
||||||
new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
|
}
|
||||||
descriptor, latch),
|
});
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.postModifyColumn(tableName, descriptor);
|
|
||||||
}
|
|
||||||
return procId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2086,87 +2121,97 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
final long nonce)
|
final long nonce)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (cpHost != null) {
|
|
||||||
if (cpHost.preDeleteColumn(tableName, columnName)) {
|
return MasterProcedureUtil.submitProcedure(
|
||||||
return -1;
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
if (getMaster().getMasterCoprocessorHost().preDeleteColumn(tableName, columnName)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
|
||||||
|
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||||
|
submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, columnName, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postDeleteColumn(tableName, columnName);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
|
|
||||||
|
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
@Override
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
protected String getDescription() {
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
return "DeleteColumnFamilyProcedure";
|
||||||
new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
|
}
|
||||||
columnName, latch),
|
});
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.postDeleteColumn(tableName, columnName);
|
|
||||||
}
|
|
||||||
return procId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long enableTable(
|
public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
|
||||||
final TableName tableName,
|
throws IOException {
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preEnableTable(tableName);
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
|
|
||||||
|
|
||||||
// Execute the operation asynchronously - client will check the progress of the operation
|
return MasterProcedureUtil.submitProcedure(
|
||||||
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
@Override
|
||||||
new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch),
|
protected void run() throws IOException {
|
||||||
nonceGroup,
|
getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
|
||||||
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).
|
|
||||||
//
|
|
||||||
// Note: if the procedure throws exception, we will catch it and rethrow.
|
|
||||||
prepareLatch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
|
||||||
cpHost.postEnableTable(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
return procId;
|
// Execute the operation asynchronously - client will check the progress of the operation
|
||||||
|
// In case the request is from a <1.1 client before returning,
|
||||||
|
// we want to make sure that the table is prepared to be
|
||||||
|
// enabled (the table is locked and the table state is set).
|
||||||
|
// Note: if the procedure throws exception, we will catch it and rethrow.
|
||||||
|
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
|
||||||
|
submitProcedure(new EnableTableProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, false, prepareLatch));
|
||||||
|
prepareLatch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postEnableTable(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "EnableTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long disableTable(
|
public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
|
||||||
final TableName tableName,
|
throws IOException {
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preDisableTable(tableName);
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
|
|
||||||
|
|
||||||
// Execute the operation asynchronously - client will check the progress of the operation
|
return MasterProcedureUtil.submitProcedure(
|
||||||
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
// Execute the operation asynchronously - client will check the progress of the operation
|
@Override
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
protected void run() throws IOException {
|
||||||
new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName, false, prepareLatch),
|
getMaster().getMasterCoprocessorHost().preDisableTable(tableName);
|
||||||
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).
|
|
||||||
//
|
|
||||||
// Note: if the procedure throws exception, we will catch it and rethrow.
|
|
||||||
prepareLatch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
|
||||||
cpHost.postDisableTable(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
return procId;
|
// Execute the operation asynchronously - client will check the progress of the operation
|
||||||
|
// In case the request is from a <1.1 client before returning,
|
||||||
|
// we want to make sure that the table is prepared to be
|
||||||
|
// enabled (the table is locked and the table state is set).
|
||||||
|
// Note: if the procedure throws exception, we will catch it and rethrow.
|
||||||
|
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
|
||||||
|
submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
tableName, false, prepareLatch));
|
||||||
|
prepareLatch.await();
|
||||||
|
|
||||||
|
getMaster().getMasterCoprocessorHost().postDisableTable(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DisableTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2207,33 +2252,56 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long modifyTable(
|
public long modifyTable(final TableName tableName, final HTableDescriptor descriptor,
|
||||||
final TableName tableName,
|
final long nonceGroup, final long nonce) throws IOException {
|
||||||
final HTableDescriptor descriptor,
|
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce)
|
|
||||||
throws IOException {
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
sanityCheckTableDescriptor(descriptor);
|
sanityCheckTableDescriptor(descriptor);
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.preModifyTable(tableName, descriptor);
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
|
return MasterProcedureUtil.submitProcedure(
|
||||||
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
getMaster().getMasterCoprocessorHost().preModifyTable(tableName, descriptor);
|
||||||
|
|
||||||
// Execute the operation synchronously - wait for the operation completes before continuing.
|
LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
|
||||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
|
||||||
long procId = this.procedureExecutor.submitProcedure(
|
|
||||||
new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor, latch),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
latch.await();
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
// Execute the operation synchronously - wait for the operation completes before continuing.
|
||||||
cpHost.postModifyTable(tableName, descriptor);
|
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||||
}
|
submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),
|
||||||
|
descriptor, latch));
|
||||||
|
latch.await();
|
||||||
|
|
||||||
return procId;
|
getMaster().getMasterCoprocessorHost().postModifyTable(tableName, descriptor);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "ModifyTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
public long restoreSnapshot(final SnapshotDescription snapshotDesc,
|
||||||
|
final long nonceGroup, final long nonce) throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
getSnapshotManager().checkSnapshotSupport();
|
||||||
|
|
||||||
|
// Ensure namespace exists. Will throw exception if non-known NS.
|
||||||
|
final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());
|
||||||
|
getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
|
||||||
|
|
||||||
|
return MasterProcedureUtil.submitProcedure(
|
||||||
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
|
@Override
|
||||||
|
protected void run() throws IOException {
|
||||||
|
setProcId(getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "RestoreSnapshotProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2460,9 +2528,11 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
|
void checkInitialized()
|
||||||
|
throws PleaseHoldException, ServerNotRunningYetException, MasterNotRunningException {
|
||||||
checkServiceStarted();
|
checkServiceStarted();
|
||||||
if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
|
if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
|
||||||
|
if (isStopped()) throw new MasterNotRunningException();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2656,18 +2726,29 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
*/
|
*/
|
||||||
long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
||||||
final long nonce)
|
final long nonce) throws IOException {
|
||||||
throws IOException {
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
|
||||||
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
||||||
if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) {
|
|
||||||
throw new BypassCoprocessorException();
|
return MasterProcedureUtil.submitProcedure(
|
||||||
}
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
|
@Override
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
protected void run() throws IOException {
|
||||||
long procId = getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
|
if (getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor)) {
|
||||||
if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
|
throw new BypassCoprocessorException();
|
||||||
return procId;
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey()));
|
||||||
|
getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "CreateTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2678,18 +2759,29 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
*/
|
*/
|
||||||
long modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
long modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
||||||
final long nonce)
|
final long nonce) throws IOException {
|
||||||
throws IOException {
|
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
|
||||||
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
||||||
if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) {
|
|
||||||
throw new BypassCoprocessorException();
|
return MasterProcedureUtil.submitProcedure(
|
||||||
}
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
|
@Override
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
protected void run() throws IOException {
|
||||||
long procId = getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
|
if (getMaster().getMasterCoprocessorHost().preModifyNamespace(namespaceDescriptor)) {
|
||||||
if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
|
throw new BypassCoprocessorException();
|
||||||
return procId;
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
setProcId(getClusterSchema().modifyNamespace(namespaceDescriptor, getNonceKey()));
|
||||||
|
getMaster().getMasterCoprocessorHost().postModifyNamespace(namespaceDescriptor);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "CreateTableProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2700,16 +2792,27 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* @return procedure id
|
* @return procedure id
|
||||||
*/
|
*/
|
||||||
long deleteNamespace(final String name, final long nonceGroup, final long nonce)
|
long deleteNamespace(final String name, final long nonceGroup, final long nonce)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) {
|
|
||||||
throw new BypassCoprocessorException();
|
return MasterProcedureUtil.submitProcedure(
|
||||||
}
|
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
|
||||||
LOG.info(getClientIdAuditPrefix() + " delete " + name);
|
@Override
|
||||||
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
protected void run() throws IOException {
|
||||||
long procId = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
|
if (getMaster().getMasterCoprocessorHost().preDeleteNamespace(name)) {
|
||||||
if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
|
throw new BypassCoprocessorException();
|
||||||
return procId;
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " delete " + name);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
setProcId(getClusterSchema().deleteNamespace(name, getNonceKey()));
|
||||||
|
getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getDescription() {
|
||||||
|
return "DeleteNamespaceProcedure";
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -1199,16 +1199,8 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
|
public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
|
||||||
RestoreSnapshotRequest request) throws ServiceException {
|
RestoreSnapshotRequest request) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
master.checkInitialized();
|
long procId = master.restoreSnapshot(request.getSnapshot(),
|
||||||
master.snapshotManager.checkSnapshotSupport();
|
request.getNonceGroup(), request.getNonce());
|
||||||
|
|
||||||
// Ensure namespace exists. Will throw exception if non-known NS.
|
|
||||||
TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
|
|
||||||
master.getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
|
|
||||||
|
|
||||||
SnapshotDescription reqSnapshot = request.getSnapshot();
|
|
||||||
long procId = master.snapshotManager.restoreOrCloneSnapshot(
|
|
||||||
reqSnapshot, request.getNonceGroup(), request.getNonce());
|
|
||||||
return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
|
return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
|
||||||
} catch (ForeignException e) {
|
} catch (ForeignException e) {
|
||||||
throw new ServiceException(e.getCause());
|
throw new ServiceException(e.getCause());
|
||||||
|
@ -1356,7 +1348,7 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
master.checkServiceStarted();
|
master.checkServiceStarted();
|
||||||
RegionStateTransition rt = req.getTransition(0);
|
RegionStateTransition rt = req.getTransition(0);
|
||||||
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
|
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
|
||||||
for (RegionInfo ri : rt.getRegionInfoList()) {
|
for (RegionInfo ri : rt.getRegionInfoList()) {
|
||||||
TableName tableName = ProtobufUtil.toTableName(ri.getTableName());
|
TableName tableName = ProtobufUtil.toTableName(ri.getTableName());
|
||||||
if (!(TableName.META_TABLE_NAME.equals(tableName)
|
if (!(TableName.META_TABLE_NAME.equals(tableName)
|
||||||
&& regionStates.getRegionState(HRegionInfo.FIRST_META_REGIONINFO) != null)
|
&& regionStates.getRegionState(HRegionInfo.FIRST_META_REGIONINFO) != null)
|
||||||
|
|
|
@ -216,10 +216,9 @@ public class TableNamespaceManager {
|
||||||
* Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
|
* Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
|
||||||
*/
|
*/
|
||||||
private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
|
private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
|
ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
|
||||||
long procId =
|
long procId = clusterSchema.createNamespace(namespaceDescriptor, null);
|
||||||
clusterSchema.createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
|
||||||
block(this.masterServices, procId);
|
block(this.masterServices, procId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,12 +18,18 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hbase.master.procedure;
|
package org.apache.hadoop.hbase.master.procedure;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
|
@ -54,4 +60,85 @@ public final class MasterProcedureUtil {
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper Runnable used in conjunction with submitProcedure() to deal with
|
||||||
|
* submitting procs with nonce.
|
||||||
|
* See submitProcedure() for an example.
|
||||||
|
*/
|
||||||
|
public static abstract class NonceProcedureRunnable {
|
||||||
|
private final MasterServices master;
|
||||||
|
private final NonceKey nonceKey;
|
||||||
|
private Long procId;
|
||||||
|
|
||||||
|
public NonceProcedureRunnable(final MasterServices master,
|
||||||
|
final long nonceGroup, final long nonce) {
|
||||||
|
this.master = master;
|
||||||
|
this.nonceKey = getProcedureExecutor().createNonceKey(nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected NonceKey getNonceKey() {
|
||||||
|
return nonceKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected MasterServices getMaster() {
|
||||||
|
return master;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ProcedureExecutor<MasterProcedureEnv> getProcedureExecutor() {
|
||||||
|
return master.getMasterProcedureExecutor();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long getProcId() {
|
||||||
|
return procId != null ? procId.longValue() : -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long setProcId(final long procId) {
|
||||||
|
this.procId = procId;
|
||||||
|
return procId;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract void run() throws IOException;
|
||||||
|
protected abstract String getDescription();
|
||||||
|
|
||||||
|
protected long submitProcedure(final Procedure proc) {
|
||||||
|
assert procId == null : "submitProcedure() was already called, running procId=" + procId;
|
||||||
|
procId = getProcedureExecutor().submitProcedure(proc, nonceKey);
|
||||||
|
return procId;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper used to deal with submitting procs with nonce.
|
||||||
|
* Internally the NonceProcedureRunnable.run() will be called only if no one else
|
||||||
|
* registered the nonce. any Exception thrown by the run() method will be
|
||||||
|
* collected/handled and rethrown.
|
||||||
|
* <code>
|
||||||
|
* long procId = MasterProcedureUtil.submitProcedure(
|
||||||
|
* new NonceProcedureRunnable(procExec, nonceGroup, nonce) {
|
||||||
|
* {@literal @}Override
|
||||||
|
* public void run() {
|
||||||
|
* cpHost.preOperation();
|
||||||
|
* submitProcedure(new MyProc());
|
||||||
|
* cpHost.postOperation();
|
||||||
|
* }
|
||||||
|
* });
|
||||||
|
* </code>
|
||||||
|
*/
|
||||||
|
public static long submitProcedure(final NonceProcedureRunnable runnable) throws IOException {
|
||||||
|
final ProcedureExecutor<MasterProcedureEnv> procExec = runnable.getProcedureExecutor();
|
||||||
|
final long procId = procExec.registerNonce(runnable.getNonceKey());
|
||||||
|
if (procId >= 0) return procId; // someone already registered the nonce
|
||||||
|
try {
|
||||||
|
runnable.run();
|
||||||
|
} catch (IOException e) {
|
||||||
|
procExec.setFailureResultForNonce(runnable.getNonceKey(),
|
||||||
|
runnable.getDescription(),
|
||||||
|
procExec.getEnvironment().getRequestUser(), e);
|
||||||
|
throw e;
|
||||||
|
} finally {
|
||||||
|
procExec.unregisterNonceIfProcedureWasNotSubmitted(runnable.getNonceKey());
|
||||||
|
}
|
||||||
|
return runnable.getProcId();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.KeyLocker;
|
import org.apache.hadoop.hbase.util.KeyLocker;
|
||||||
|
import org.apache.hadoop.hbase.util.NonceKey;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -674,18 +675,13 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
* @param tableName table to clone
|
* @param tableName table to clone
|
||||||
* @param snapshot Snapshot Descriptor
|
* @param snapshot Snapshot Descriptor
|
||||||
* @param snapshotTableDesc Table Descriptor
|
* @param snapshotTableDesc Table Descriptor
|
||||||
* @param nonceGroup unique value to prevent duplicated RPC
|
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||||
* @param nonce unique value to prevent duplicated RPC
|
|
||||||
* @return procId the ID of the clone snapshot procedure
|
* @return procId the ID of the clone snapshot procedure
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private long cloneSnapshot(
|
private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
|
||||||
final SnapshotDescription reqSnapshot,
|
final SnapshotDescription snapshot, final HTableDescriptor snapshotTableDesc,
|
||||||
final TableName tableName,
|
final NonceKey nonceKey) throws IOException {
|
||||||
final SnapshotDescription snapshot,
|
|
||||||
final HTableDescriptor snapshotTableDesc,
|
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
||||||
HTableDescriptor htd = new HTableDescriptor(tableName, snapshotTableDesc);
|
HTableDescriptor htd = new HTableDescriptor(tableName, snapshotTableDesc);
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
|
@ -693,7 +689,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
}
|
}
|
||||||
long procId;
|
long procId;
|
||||||
try {
|
try {
|
||||||
procId = cloneSnapshot(snapshot, htd, nonceGroup, nonce);
|
procId = cloneSnapshot(snapshot, htd, nonceKey);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName()
|
LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName()
|
||||||
+ " as table " + tableName.getNameAsString(), e);
|
+ " as table " + tableName.getNameAsString(), e);
|
||||||
|
@ -713,15 +709,12 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
*
|
*
|
||||||
* @param snapshot Snapshot Descriptor
|
* @param snapshot Snapshot Descriptor
|
||||||
* @param hTableDescriptor Table Descriptor of the table to create
|
* @param hTableDescriptor Table Descriptor of the table to create
|
||||||
* @param nonceGroup unique value to prevent duplicated RPC
|
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||||
* @param nonce unique value to prevent duplicated RPC
|
|
||||||
* @return procId the ID of the clone snapshot procedure
|
* @return procId the ID of the clone snapshot procedure
|
||||||
*/
|
*/
|
||||||
synchronized long cloneSnapshot(
|
synchronized long cloneSnapshot(final SnapshotDescription snapshot,
|
||||||
final SnapshotDescription snapshot,
|
final HTableDescriptor hTableDescriptor, final NonceKey nonceKey)
|
||||||
final HTableDescriptor hTableDescriptor,
|
throws HBaseSnapshotException {
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws HBaseSnapshotException {
|
|
||||||
TableName tableName = hTableDescriptor.getTableName();
|
TableName tableName = hTableDescriptor.getTableName();
|
||||||
|
|
||||||
// make sure we aren't running a snapshot on the same table
|
// make sure we aren't running a snapshot on the same table
|
||||||
|
@ -738,8 +731,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
||||||
new CloneSnapshotProcedure(
|
new CloneSnapshotProcedure(
|
||||||
master.getMasterProcedureExecutor().getEnvironment(), hTableDescriptor, snapshot),
|
master.getMasterProcedureExecutor().getEnvironment(), hTableDescriptor, snapshot),
|
||||||
nonceGroup,
|
nonceKey);
|
||||||
nonce);
|
|
||||||
this.restoreTableToProcIdMap.put(tableName, procId);
|
this.restoreTableToProcIdMap.put(tableName, procId);
|
||||||
return procId;
|
return procId;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
@ -753,14 +745,11 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
/**
|
/**
|
||||||
* Restore or Clone the specified snapshot
|
* Restore or Clone the specified snapshot
|
||||||
* @param reqSnapshot
|
* @param reqSnapshot
|
||||||
* @param nonceGroup unique value to prevent duplicated RPC
|
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||||
* @param nonce unique value to prevent duplicated RPC
|
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public long restoreOrCloneSnapshot(
|
public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final NonceKey nonceKey)
|
||||||
SnapshotDescription reqSnapshot,
|
throws IOException {
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
||||||
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
|
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
|
||||||
|
|
||||||
|
@ -789,11 +778,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
// Execute the restore/clone operation
|
// Execute the restore/clone operation
|
||||||
long procId;
|
long procId;
|
||||||
if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
|
if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
|
||||||
procId = restoreSnapshot(
|
procId = restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey);
|
||||||
reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceGroup, nonce);
|
|
||||||
} else {
|
} else {
|
||||||
procId = cloneSnapshot(
|
procId = cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey);
|
||||||
reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceGroup, nonce);
|
|
||||||
}
|
}
|
||||||
return procId;
|
return procId;
|
||||||
}
|
}
|
||||||
|
@ -806,18 +793,13 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
* @param tableName table to restore
|
* @param tableName table to restore
|
||||||
* @param snapshot Snapshot Descriptor
|
* @param snapshot Snapshot Descriptor
|
||||||
* @param snapshotTableDesc Table Descriptor
|
* @param snapshotTableDesc Table Descriptor
|
||||||
* @param nonceGroup unique value to prevent duplicated RPC
|
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||||
* @param nonce unique value to prevent duplicated RPC
|
|
||||||
* @return procId the ID of the restore snapshot procedure
|
* @return procId the ID of the restore snapshot procedure
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private long restoreSnapshot(
|
private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
|
||||||
final SnapshotDescription reqSnapshot,
|
final SnapshotDescription snapshot, final HTableDescriptor snapshotTableDesc,
|
||||||
final TableName tableName,
|
final NonceKey nonceKey) throws IOException {
|
||||||
final SnapshotDescription snapshot,
|
|
||||||
final HTableDescriptor snapshotTableDesc,
|
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
|
||||||
|
|
||||||
if (master.getTableStateManager().isTableState(
|
if (master.getTableStateManager().isTableState(
|
||||||
|
@ -834,7 +816,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
|
|
||||||
long procId;
|
long procId;
|
||||||
try {
|
try {
|
||||||
procId = restoreSnapshot(snapshot, snapshotTableDesc, nonceGroup, nonce);
|
procId = restoreSnapshot(snapshot, snapshotTableDesc, nonceKey);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Exception occurred while restoring the snapshot " + snapshot.getName()
|
LOG.error("Exception occurred while restoring the snapshot " + snapshot.getName()
|
||||||
+ " as table " + tableName.getNameAsString(), e);
|
+ " as table " + tableName.getNameAsString(), e);
|
||||||
|
@ -855,16 +837,13 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
*
|
*
|
||||||
* @param snapshot Snapshot Descriptor
|
* @param snapshot Snapshot Descriptor
|
||||||
* @param hTableDescriptor Table Descriptor
|
* @param hTableDescriptor Table Descriptor
|
||||||
* @param nonceGroup unique value to prevent duplicated RPC
|
* @param nonceKey unique identifier to prevent duplicated RPC
|
||||||
* @param nonce unique value to prevent duplicated RPC
|
|
||||||
* @return procId the ID of the restore snapshot procedure
|
* @return procId the ID of the restore snapshot procedure
|
||||||
*/
|
*/
|
||||||
private synchronized long restoreSnapshot(
|
private synchronized long restoreSnapshot(final SnapshotDescription snapshot,
|
||||||
final SnapshotDescription snapshot,
|
final HTableDescriptor hTableDescriptor, final NonceKey nonceKey)
|
||||||
final HTableDescriptor hTableDescriptor,
|
throws HBaseSnapshotException {
|
||||||
final long nonceGroup,
|
final TableName tableName = hTableDescriptor.getTableName();
|
||||||
final long nonce) throws HBaseSnapshotException {
|
|
||||||
TableName tableName = hTableDescriptor.getTableName();
|
|
||||||
|
|
||||||
// make sure we aren't running a snapshot on the same table
|
// make sure we aren't running a snapshot on the same table
|
||||||
if (isTakingSnapshot(tableName)) {
|
if (isTakingSnapshot(tableName)) {
|
||||||
|
@ -880,8 +859,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
||||||
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
long procId = master.getMasterProcedureExecutor().submitProcedure(
|
||||||
new RestoreSnapshotProcedure(
|
new RestoreSnapshotProcedure(
|
||||||
master.getMasterProcedureExecutor().getEnvironment(), hTableDescriptor, snapshot),
|
master.getMasterProcedureExecutor().getEnvironment(), hTableDescriptor, snapshot),
|
||||||
nonceGroup,
|
nonceKey);
|
||||||
nonce);
|
|
||||||
this.restoreTableToProcIdMap.put(tableName, procId);
|
this.restoreTableToProcIdMap.put(tableName, procId);
|
||||||
return procId;
|
return procId;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|
|
@ -51,9 +51,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Test 1: Add a column family online
|
// Test 1: Add a column family online
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor1),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor1));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -64,9 +62,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Test 2: Add a column family offline
|
// Test 2: Add a column family offline
|
||||||
UTIL.getHBaseAdmin().disableTable(tableName);
|
UTIL.getHBaseAdmin().disableTable(tableName);
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor2));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
||||||
|
@ -86,9 +82,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// add the column family
|
// add the column family
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -97,9 +91,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// add the column family that exists
|
// add the column family that exists
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
|
|
||||||
|
@ -113,9 +105,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Do the same add the existing column family - this time offline
|
// Do the same add the existing column family - this time offline
|
||||||
UTIL.getHBaseAdmin().disableTable(tableName);
|
UTIL.getHBaseAdmin().disableTable(tableName);
|
||||||
long procId3 = procExec.submitProcedure(
|
long procId3 = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup + 2,
|
|
||||||
nonce + 2);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId3);
|
ProcedureTestingUtility.waitProcedure(procExec, procId3);
|
||||||
|
|
||||||
|
@ -127,37 +117,6 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
|
ProcedureTestingUtility.getExceptionCause(result) 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)
|
@Test(timeout = 60000)
|
||||||
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
|
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
|
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
|
||||||
|
@ -173,9 +132,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the AddColumnFamily procedure && kill the executor
|
// Start the AddColumnFamily procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = AddColumnFamilyState.values().length;
|
int numberOfSteps = AddColumnFamilyState.values().length;
|
||||||
|
@ -199,9 +156,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the AddColumnFamily procedure && kill the executor
|
// Start the AddColumnFamily procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = AddColumnFamilyState.values().length;
|
int numberOfSteps = AddColumnFamilyState.values().length;
|
||||||
|
@ -225,9 +180,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the AddColumnFamily procedure && kill the executor
|
// Start the AddColumnFamily procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 1; // failing at "pre operations"
|
int numberOfSteps = 1; // failing at "pre operations"
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -118,29 +118,6 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
|
||||||
clonedTableName);
|
clonedTableName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
|
||||||
public void testCloneSnapshotTwiceWithSameNonce() throws Exception {
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
final TableName clonedTableName = TableName.valueOf("testCloneSnapshotTwiceWithSameNonce");
|
|
||||||
final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
|
|
||||||
|
|
||||||
// take the snapshot
|
|
||||||
HBaseProtos.SnapshotDescription snapshotDesc = getSnapshot();
|
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
|
||||||
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc), nonceGroup, nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(
|
|
||||||
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc), 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)
|
@Test(timeout=60000)
|
||||||
public void testCloneSnapshotToSameTable() throws Exception {
|
public void testCloneSnapshotToSameTable() throws Exception {
|
||||||
// take the snapshot
|
// take the snapshot
|
||||||
|
@ -172,7 +149,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Clone snapshot procedure && kill the executor
|
// Start the Clone snapshot procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc), nonceGroup, nonce);
|
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = CloneSnapshotState.values().length;
|
int numberOfSteps = CloneSnapshotState.values().length;
|
||||||
|
@ -197,7 +174,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Clone snapshot procedure && kill the executor
|
// Start the Clone snapshot procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc), nonceGroup, nonce);
|
new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
|
||||||
|
|
||||||
int numberOfSteps = 0; // failing at pre operation
|
int numberOfSteps = 0; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -52,9 +52,6 @@ public class TestCreateNamespaceProcedure {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
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) {
|
private static void setupConf(Configuration conf) {
|
||||||
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
||||||
}
|
}
|
||||||
|
@ -77,9 +74,6 @@ public class TestCreateNamespaceProcedure {
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -93,9 +87,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -110,18 +102,14 @@ public class TestCreateNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
|
||||||
// Create the namespace that exists
|
// Create the namespace that exists
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
|
|
||||||
|
@ -140,9 +128,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -163,9 +149,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
nsd.setConfiguration(nsKey, nsValue);
|
nsd.setConfiguration(nsKey, nsValue);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -185,9 +169,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
nsd.setConfiguration(nsKey, nsValue);
|
nsd.setConfiguration(nsKey, nsValue);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -196,32 +178,6 @@ public class TestCreateNamespaceProcedure {
|
||||||
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
|
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testCreateSameNamespaceTwiceWithSameNonce() throws Exception {
|
|
||||||
final NamespaceDescriptor nsd =
|
|
||||||
NamespaceDescriptor.create("testCreateSameNamespaceTwiceWithSameNonce").build();
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(
|
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
|
||||||
|
|
||||||
validateNamespaceCreated(nsd);
|
|
||||||
|
|
||||||
// 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)
|
@Test(timeout = 60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final NamespaceDescriptor nsd =
|
final NamespaceDescriptor nsd =
|
||||||
|
@ -233,9 +189,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
|
|
||||||
// Start the CreateNamespace procedure && kill the executor
|
// Start the CreateNamespace procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = CreateNamespaceState.values().length;
|
int numberOfSteps = CreateNamespaceState.values().length;
|
||||||
|
@ -257,9 +211,7 @@ public class TestCreateNamespaceProcedure {
|
||||||
|
|
||||||
// Start the CreateNamespace procedure && kill the executor
|
// Start the CreateNamespace procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 0; // failing at pre operation
|
int numberOfSteps = 0; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -92,14 +92,12 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// create the table
|
// create the table
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
|
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||||
|
|
||||||
// create another with the same name
|
// create another with the same name
|
||||||
ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch();
|
ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch();
|
||||||
long procId2 = procExec.submitProcedure(
|
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.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
|
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
|
||||||
|
@ -108,29 +106,6 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
||||||
latch2.await();
|
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)
|
@Test(timeout=60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
||||||
|
@ -144,7 +119,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
||||||
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
|
||||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
|
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
// NOTE: the 6 (number of CreateTableState steps) is hardcoded,
|
// NOTE: the 6 (number of CreateTableState steps) is hardcoded,
|
||||||
|
@ -181,7 +156,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
||||||
htd.setRegionReplication(3);
|
htd.setRegionReplication(3);
|
||||||
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateTableProcedure(procExec.getEnvironment(), htd, regions), nonceGroup, nonce);
|
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
|
||||||
|
|
||||||
int numberOfSteps = 0; // failing at pre operation
|
int numberOfSteps = 0; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -53,9 +53,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Test 1: delete the column family that exists online
|
// Test 1: delete the column family that exists online
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf1.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf1.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -66,9 +64,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Test 2: delete the column family that exists offline
|
// Test 2: delete the column family that exists offline
|
||||||
UTIL.getHBaseAdmin().disableTable(tableName);
|
UTIL.getHBaseAdmin().disableTable(tableName);
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
||||||
|
@ -85,9 +81,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// delete the column family that exists
|
// delete the column family that exists
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
// First delete should succeed
|
// First delete should succeed
|
||||||
|
@ -98,9 +92,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// delete the column family that does not exist
|
// delete the column family that does not exist
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
|
@ -115,9 +107,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Try again, this time with table disabled.
|
// Try again, this time with table disabled.
|
||||||
UTIL.getHBaseAdmin().disableTable(tableName);
|
UTIL.getHBaseAdmin().disableTable(tableName);
|
||||||
long procId3 = procExec.submitProcedure(
|
long procId3 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf2.getBytes()));
|
||||||
nonceGroup + 2,
|
|
||||||
nonce + 2);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId3);
|
ProcedureTestingUtility.waitProcedure(procExec, procId3);
|
||||||
// Expect fail with InvalidFamilyOperationException
|
// Expect fail with InvalidFamilyOperationException
|
||||||
|
@ -128,37 +118,6 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
|
ProcedureTestingUtility.getExceptionCause(result) 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)
|
@Test(timeout=60000)
|
||||||
public void testDeleteNonExistingColumnFamily() throws Exception {
|
public void testDeleteNonExistingColumnFamily() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily");
|
final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily");
|
||||||
|
@ -170,9 +129,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// delete the column family that does not exist
|
// delete the column family that does not exist
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf3.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf3.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
|
|
||||||
|
@ -198,9 +155,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Delete procedure && kill the executor
|
// Start the Delete procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = DeleteColumnFamilyState.values().length;
|
int numberOfSteps = DeleteColumnFamilyState.values().length;
|
||||||
|
@ -224,9 +179,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Delete procedure && kill the executor
|
// Start the Delete procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = DeleteColumnFamilyState.values().length;
|
int numberOfSteps = DeleteColumnFamilyState.values().length;
|
||||||
|
@ -251,9 +204,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Delete procedure && kill the executor
|
// Start the Delete procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()),
|
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 1; // failing at pre operation
|
int numberOfSteps = 1; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -53,9 +53,6 @@ public class TestDeleteNamespaceProcedure {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
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) {
|
private static void setupConf(Configuration conf) {
|
||||||
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
||||||
}
|
}
|
||||||
|
@ -78,9 +75,6 @@ public class TestDeleteNamespaceProcedure {
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -100,9 +94,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
createNamespaceForTesting(namespaceName);
|
createNamespaceForTesting(namespaceName);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -118,9 +110,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
validateNamespaceNotExist(namespaceName);
|
validateNamespaceNotExist(namespaceName);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
// Expect fail with NamespaceNotFoundException
|
// Expect fail with NamespaceNotFoundException
|
||||||
|
@ -137,9 +127,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -159,9 +147,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
|
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -170,33 +156,6 @@ public class TestDeleteNamespaceProcedure {
|
||||||
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
|
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testDeleteSameNamespaceTwiceWithSameNonce() throws Exception {
|
|
||||||
final String namespaceName = "testDeleteSameNamespaceTwiceWithSameNonce";
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
createNamespaceForTesting(namespaceName);
|
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(
|
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
|
||||||
|
|
||||||
validateNamespaceNotExist(namespaceName);
|
|
||||||
|
|
||||||
// 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)
|
@Test(timeout = 60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final String namespaceName = "testRecoveryAndDoubleExecution";
|
final String namespaceName = "testRecoveryAndDoubleExecution";
|
||||||
|
@ -209,9 +168,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
|
|
||||||
// Start the DeleteNamespace procedure && kill the executor
|
// Start the DeleteNamespace procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = DeleteNamespaceState.values().length;
|
int numberOfSteps = DeleteNamespaceState.values().length;
|
||||||
|
@ -233,11 +190,8 @@ public class TestDeleteNamespaceProcedure {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||||
|
|
||||||
// Start the DeleteNamespace procedure && kill the executor
|
// Start the DeleteNamespace procedure && kill the executor
|
||||||
LOG.info("SUBMIT DELTET");
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
|
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 0; // failing at pre operation
|
int numberOfSteps = 0; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
@ -257,9 +211,7 @@ public class TestDeleteNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
|
|
@ -74,10 +74,10 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// delete the table (that exists)
|
// delete the table (that exists)
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
|
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
|
||||||
// delete the table (that will no longer exist)
|
// delete the table (that will no longer exist)
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup + 1, nonce + 1);
|
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
|
||||||
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
|
@ -95,36 +95,6 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
|
||||||
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotFoundException);
|
assertTrue(ProcedureTestingUtility.getExceptionCause(result) 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);
|
|
||||||
|
|
||||||
// Second delete should not fail, because it is the same delete
|
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
|
||||||
assertTrue(procId1 == procId2);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testSimpleDelete() throws Exception {
|
public void testSimpleDelete() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testSimpleDelete");
|
final TableName tableName = TableName.valueOf("testSimpleDelete");
|
||||||
|
@ -171,7 +141,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Delete procedure && kill the executor
|
// Start the Delete procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
|
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
// NOTE: the 6 (number of DeleteTableState steps) is hardcoded,
|
// NOTE: the 6 (number of DeleteTableState steps) is hardcoded,
|
||||||
|
|
|
@ -48,7 +48,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Disable the table
|
// Disable the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -65,7 +65,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Disable the table
|
// Disable the table
|
||||||
long procId1 = procExec.submitProcedure(new DisableTableProcedure(
|
long procId1 = procExec.submitProcedure(new DisableTableProcedure(
|
||||||
procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
procExec.getEnvironment(), tableName, false));
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -74,7 +74,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Disable the table again - expect failure
|
// Disable the table again - expect failure
|
||||||
long procId2 = procExec.submitProcedure(new DisableTableProcedure(
|
long procId2 = procExec.submitProcedure(new DisableTableProcedure(
|
||||||
procExec.getEnvironment(), tableName, false), nonceGroup + 1, nonce + 1);
|
procExec.getEnvironment(), tableName, false));
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
ProcedureInfo result = procExec.getResult(procId2);
|
ProcedureInfo result = procExec.getResult(procId2);
|
||||||
|
@ -88,7 +88,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
|
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
|
||||||
|
|
||||||
long procId3 = procExec.submitProcedure(new DisableTableProcedure(
|
long procId3 = procExec.submitProcedure(new DisableTableProcedure(
|
||||||
procExec.getEnvironment(), tableName, false, prepareLatch), nonceGroup + 2, nonce + 2);
|
procExec.getEnvironment(), tableName, false, prepareLatch));
|
||||||
prepareLatch.await();
|
prepareLatch.await();
|
||||||
Assert.fail("Disable should throw exception through latch.");
|
Assert.fail("Disable should throw exception through latch.");
|
||||||
} catch (TableNotEnabledException tnee) {
|
} catch (TableNotEnabledException tnee) {
|
||||||
|
@ -106,29 +106,6 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
tableName);
|
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)
|
@Test(timeout=60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
||||||
|
@ -143,7 +120,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Disable procedure && kill the executor
|
// Start the Disable procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = DisableTableState.values().length;
|
int numberOfSteps = DisableTableState.values().length;
|
||||||
|
|
|
@ -54,8 +54,6 @@ public class TestDispatchMergingRegionsProcedure {
|
||||||
private static final Log LOG = LogFactory.getLog(TestDispatchMergingRegionsProcedure.class);
|
private static final Log LOG = LogFactory.getLog(TestDispatchMergingRegionsProcedure.class);
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||||
private static long nonceGroup = HConstants.NO_NONCE;
|
|
||||||
private static long nonce = HConstants.NO_NONCE;
|
|
||||||
|
|
||||||
private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
|
private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
|
||||||
final static Configuration conf = UTIL.getConfiguration();
|
final static Configuration conf = UTIL.getConfiguration();
|
||||||
|
@ -89,9 +87,7 @@ public class TestDispatchMergingRegionsProcedure {
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
resetProcExecutorTestingKillFlag();
|
resetProcExecutorTestingKillFlag();
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
// Turn off balancer so it doesn't cut in and mess up our placements.
|
// Turn off balancer so it doesn't cut in and mess up our placements.
|
||||||
UTIL.getHBaseAdmin().setBalancerRunning(false, true);
|
UTIL.getHBaseAdmin().setBalancerRunning(false, true);
|
||||||
// Turn off the meta scanner so it don't remove parent on us.
|
// Turn off the meta scanner so it don't remove parent on us.
|
||||||
|
@ -187,33 +183,6 @@ public class TestDispatchMergingRegionsProcedure {
|
||||||
return completedTaskCount;
|
return completedTaskCount;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testMergeRegionsTwiceWithSameNonce() throws Exception {
|
|
||||||
final TableName tableName = TableName.valueOf("testMergeRegionsTwiceWithSameNonce");
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
List<HRegionInfo> tableRegions = createTable(tableName, 3);
|
|
||||||
|
|
||||||
HRegionInfo[] regionsToMerge = new HRegionInfo[2];
|
|
||||||
regionsToMerge[0] = tableRegions.get(0);
|
|
||||||
regionsToMerge[1] = tableRegions.get(1);
|
|
||||||
|
|
||||||
final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
|
|
||||||
long procId1 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
|
|
||||||
procExec.getEnvironment(), tableName, regionsToMerge, true), nonceGroup, nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
|
|
||||||
procExec.getEnvironment(), tableName, regionsToMerge, true), nonceGroup, nonce);
|
|
||||||
assertEquals(procId1, procId2);
|
|
||||||
|
|
||||||
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);
|
|
||||||
|
|
||||||
assertRegionCount(tableName, 2, 1, initCompletedTaskCount);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
||||||
|
|
|
@ -49,7 +49,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Enable the table
|
// Enable the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -57,29 +57,6 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
tableName);
|
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)
|
@Test(timeout=60000, expected=TableNotDisabledException.class)
|
||||||
public void testEnableNonDisabledTable() throws Exception {
|
public void testEnableNonDisabledTable() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testEnableNonExistingTable");
|
final TableName tableName = TableName.valueOf("testEnableNonExistingTable");
|
||||||
|
@ -89,7 +66,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Enable the table - expect failure
|
// Enable the table - expect failure
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
|
|
||||||
ProcedureInfo result = procExec.getResult(procId1);
|
ProcedureInfo result = procExec.getResult(procId1);
|
||||||
|
@ -100,9 +77,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Enable the table with skipping table state check flag (simulate recovery scenario)
|
// Enable the table with skipping table state check flag (simulate recovery scenario)
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, true),
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, true));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
||||||
|
@ -110,9 +85,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
// Enable the table - expect failure from ProcedurePrepareLatch
|
// Enable the table - expect failure from ProcedurePrepareLatch
|
||||||
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
|
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
|
||||||
long procId3 = procExec.submitProcedure(
|
long procId3 = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch),
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch));
|
||||||
nonceGroup + 2,
|
|
||||||
nonce + 2);
|
|
||||||
prepareLatch.await();
|
prepareLatch.await();
|
||||||
Assert.fail("Enable should throw exception through latch.");
|
Assert.fail("Enable should throw exception through latch.");
|
||||||
}
|
}
|
||||||
|
@ -132,7 +105,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Enable procedure && kill the executor
|
// Start the Enable procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = EnableTableState.values().length;
|
int numberOfSteps = EnableTableState.values().length;
|
||||||
|
@ -156,7 +129,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Enable procedure && kill the executor
|
// Start the Enable procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new EnableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
|
|
||||||
int numberOfSteps = 1; // failing at pre operation
|
int numberOfSteps = 1; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -53,9 +53,6 @@ public class TestMasterProcedureEvents {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
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) {
|
private static void setupConf(Configuration conf) {
|
||||||
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
||||||
conf.setBoolean(WALProcedureStore.USE_HSYNC_CONF_KEY, false);
|
conf.setBoolean(WALProcedureStore.USE_HSYNC_CONF_KEY, false);
|
||||||
|
@ -156,7 +153,7 @@ public class TestMasterProcedureEvents {
|
||||||
|
|
||||||
// submit the procedure
|
// submit the procedure
|
||||||
LOG.debug("submit " + proc);
|
LOG.debug("submit " + proc);
|
||||||
long procId = procExec.submitProcedure(proc, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
long procId = procExec.submitProcedure(proc);
|
||||||
|
|
||||||
// wait until the event is in the queue (proc executed and got into suspended state)
|
// wait until the event is in the queue (proc executed and got into suspended state)
|
||||||
LOG.debug("wait procedure suspended on " + event);
|
LOG.debug("wait procedure suspended on " + event);
|
||||||
|
|
|
@ -162,32 +162,6 @@ public class TestMergeTableRegionsProcedure {
|
||||||
assertRegionCount(tableName, initialRegionCount - 2);
|
assertRegionCount(tableName, initialRegionCount - 2);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testMergeRegionsTwiceWithSameNonce() throws Exception {
|
|
||||||
final TableName tableName = TableName.valueOf("testMergeRegionsTwiceWithSameNonce");
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
List<HRegionInfo> tableRegions = createTable(tableName);
|
|
||||||
|
|
||||||
HRegionInfo[] regionsToMerge = new HRegionInfo[2];
|
|
||||||
regionsToMerge[0] = tableRegions.get(0);
|
|
||||||
regionsToMerge[1] = tableRegions.get(1);
|
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(new MergeTableRegionsProcedure(
|
|
||||||
procExec.getEnvironment(), regionsToMerge, true), nonceGroup, nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(new MergeTableRegionsProcedure(
|
|
||||||
procExec.getEnvironment(), regionsToMerge, true), nonceGroup, nonce);
|
|
||||||
assertEquals(procId1, procId2);
|
|
||||||
|
|
||||||
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);
|
|
||||||
|
|
||||||
assertRegionCount(tableName, initialRegionCount - 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testRecoveryAndDoubleExecution() throws Exception {
|
public void testRecoveryAndDoubleExecution() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
|
||||||
|
|
|
@ -53,9 +53,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Test 1: modify the column family online
|
// Test 1: modify the column family online
|
||||||
columnDescriptor.setBlocksize(newBlockSize);
|
columnDescriptor.setBlocksize(newBlockSize);
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -66,9 +64,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
UTIL.getHBaseAdmin().disableTable(tableName);
|
UTIL.getHBaseAdmin().disableTable(tableName);
|
||||||
columnDescriptor.setBlocksize(newBlockSize * 2);
|
columnDescriptor.setBlocksize(newBlockSize * 2);
|
||||||
long procId2 = procExec.submitProcedure(
|
long procId2 = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
ProcedureTestingUtility.waitProcedure(procExec, procId2);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
|
||||||
|
@ -91,9 +87,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Modify the column family that does not exist
|
// Modify the column family that does not exist
|
||||||
columnDescriptor.setBlocksize(newBlockSize);
|
columnDescriptor.setBlocksize(newBlockSize);
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
|
|
||||||
|
@ -123,9 +117,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
columnDescriptor.setBlocksize(newBlockSize);
|
columnDescriptor.setBlocksize(newBlockSize);
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = ModifyColumnFamilyState.values().length;
|
int numberOfSteps = ModifyColumnFamilyState.values().length;
|
||||||
|
@ -153,9 +145,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
columnDescriptor.setBlocksize(newBlockSize);
|
columnDescriptor.setBlocksize(newBlockSize);
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = ModifyColumnFamilyState.values().length;
|
int numberOfSteps = ModifyColumnFamilyState.values().length;
|
||||||
|
@ -183,9 +173,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
columnDescriptor.setBlocksize(newBlockSize);
|
columnDescriptor.setBlocksize(newBlockSize);
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor),
|
new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 1; // failing at pre operation
|
int numberOfSteps = 1; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
|
|
@ -49,9 +49,6 @@ public class TestModifyNamespaceProcedure {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
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) {
|
private static void setupConf(Configuration conf) {
|
||||||
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
||||||
}
|
}
|
||||||
|
@ -74,9 +71,6 @@ public class TestModifyNamespaceProcedure {
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -113,9 +107,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
nsd.setConfiguration(nsKey2, nsValue2);
|
nsd.setConfiguration(nsKey2, nsValue2);
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
|
||||||
|
@ -143,9 +135,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
|
final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
|
|
||||||
|
@ -171,9 +161,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
nsd.setConfiguration(nsKey, nsValue);
|
nsd.setConfiguration(nsKey, nsValue);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -196,9 +184,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
nsd.setConfiguration(nsKey, nsValue);
|
nsd.setConfiguration(nsKey, nsValue);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureInfo result = procExec.getResult(procId);
|
ProcedureInfo result = procExec.getResult(procId);
|
||||||
|
@ -224,9 +210,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = ModifyNamespaceState.values().length;
|
int numberOfSteps = ModifyNamespaceState.values().length;
|
||||||
|
@ -256,9 +240,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
|
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
int numberOfSteps = 0; // failing at pre operation
|
int numberOfSteps = 0; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
@ -277,9 +259,7 @@ public class TestModifyNamespaceProcedure {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new CreateNamespaceProcedure(procExec.getEnvironment(), nsDescriptor),
|
new CreateNamespaceProcedure(procExec.getEnvironment(), nsDescriptor));
|
||||||
nonceGroup + 1,
|
|
||||||
nonce + 1);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
|
|
@ -201,7 +201,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
|
new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = ModifyTableState.values().length;
|
int numberOfSteps = ModifyTableState.values().length;
|
||||||
|
@ -239,7 +239,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
|
new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = ModifyTableState.values().length;
|
int numberOfSteps = ModifyTableState.values().length;
|
||||||
|
@ -276,7 +276,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
|
new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||||
|
|
||||||
int numberOfSteps = 1; // failing at pre operation
|
int numberOfSteps = 1; // failing at pre operation
|
||||||
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
|
||||||
|
@ -308,7 +308,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Modify procedure && kill the executor
|
// Start the Modify procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new ModifyTableProcedure(procExec.getEnvironment(), htd), nonceGroup, nonce);
|
new ModifyTableProcedure(procExec.getEnvironment(), htd));
|
||||||
|
|
||||||
// Restart the executor and rollback the step twice
|
// Restart the executor and rollback the step twice
|
||||||
int numberOfSteps = 1; // failing at pre operation
|
int numberOfSteps = 1; // failing at pre operation
|
||||||
|
|
|
@ -50,9 +50,6 @@ public class TestProcedureAdmin {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
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) {
|
private static void setupConf(Configuration conf) {
|
||||||
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
|
||||||
}
|
}
|
||||||
|
@ -77,10 +74,6 @@ public class TestProcedureAdmin {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
|
||||||
assertTrue("expected executor to be running", procExec.isRunning());
|
assertTrue("expected executor to be running", procExec.isRunning());
|
||||||
|
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -103,7 +96,7 @@ public class TestProcedureAdmin {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||||
// Submit an abortable procedure
|
// Submit an abortable procedure
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
// Wait for one step to complete
|
// Wait for one step to complete
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
|
|
||||||
|
@ -130,7 +123,7 @@ public class TestProcedureAdmin {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||||
// Submit an un-abortable procedure
|
// Submit an un-abortable procedure
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DeleteTableProcedure(procExec.getEnvironment(), tableName), nonceGroup, nonce);
|
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
|
||||||
// Wait for a couple of steps to complete (first step "prepare" is abortable)
|
// Wait for a couple of steps to complete (first step "prepare" is abortable)
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
for (int i = 0; i < 2; ++i) {
|
for (int i = 0; i < 2; ++i) {
|
||||||
|
@ -161,7 +154,7 @@ public class TestProcedureAdmin {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||||
// Submit a procedure
|
// Submit a procedure
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DisableTableProcedure(procExec.getEnvironment(), tableName, true), nonceGroup, nonce);
|
new DisableTableProcedure(procExec.getEnvironment(), tableName, true));
|
||||||
// Wait for one step to complete
|
// Wait for one step to complete
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
|
|
||||||
|
@ -202,7 +195,7 @@ public class TestProcedureAdmin {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
|
||||||
|
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new DisableTableProcedure(procExec.getEnvironment(), tableName, false), nonceGroup, nonce);
|
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
|
||||||
// Wait for one step to complete
|
// Wait for one step to complete
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
|
|
||||||
|
|
|
@ -151,30 +151,6 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
|
||||||
validateSnapshotRestore();
|
validateSnapshotRestore();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
|
||||||
public void testRestoreSnapshotTwiceWithSameNonce() throws Exception {
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
long procId1 = procExec.submitProcedure(
|
|
||||||
new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
long procId2 = procExec.submitProcedure(
|
|
||||||
new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot),
|
|
||||||
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);
|
|
||||||
|
|
||||||
validateSnapshotRestore();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testRestoreSnapshotToDifferentTable() throws Exception {
|
public void testRestoreSnapshotToDifferentTable() throws Exception {
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
||||||
|
@ -218,9 +194,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Restore snapshot procedure && kill the executor
|
// Start the Restore snapshot procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot),
|
new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = RestoreSnapshotState.values().length;
|
int numberOfSteps = RestoreSnapshotState.values().length;
|
||||||
|
|
|
@ -63,9 +63,6 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||||
|
|
||||||
private static long nonceGroup = HConstants.NO_NONCE;
|
|
||||||
private static long nonce = HConstants.NO_NONCE;
|
|
||||||
|
|
||||||
private static String ColumnFamilyName1 = "cf1";
|
private static String ColumnFamilyName1 = "cf1";
|
||||||
private static String ColumnFamilyName2 = "cf2";
|
private static String ColumnFamilyName2 = "cf2";
|
||||||
|
|
||||||
|
@ -95,9 +92,6 @@ public class TestSplitTableRegionProcedure {
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
|
||||||
nonceGroup =
|
|
||||||
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
|
|
||||||
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
|
|
||||||
|
|
||||||
// Turn off balancer so it doesn't cut in and mess up our placements.
|
// Turn off balancer so it doesn't cut in and mess up our placements.
|
||||||
UTIL.getHBaseAdmin().setBalancerRunning(false, true);
|
UTIL.getHBaseAdmin().setBalancerRunning(false, true);
|
||||||
|
@ -130,9 +124,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -155,9 +147,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -183,9 +173,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -210,9 +198,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -242,9 +228,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Wait the completion
|
// Wait the completion
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
ProcedureTestingUtility.waitProcedure(procExec, procId);
|
||||||
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
|
||||||
|
@ -266,42 +250,6 @@ public class TestSplitTableRegionProcedure {
|
||||||
assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
|
assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testSplitTableRegionTwiceWithSameNonce() throws Exception {
|
|
||||||
final TableName tableName = TableName.valueOf("testSplitTableRegionTwiceWithSameNonce");
|
|
||||||
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
|
|
||||||
|
|
||||||
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
|
|
||||||
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
|
|
||||||
insertData(tableName);
|
|
||||||
int splitRowNum = startRowNum + rowCount / 2;
|
|
||||||
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
|
|
||||||
|
|
||||||
assertTrue("not able to find a splittable region", regions != null);
|
|
||||||
assertTrue("not able to find a splittable region", regions.length == 1);
|
|
||||||
|
|
||||||
// Split region of the table
|
|
||||||
long procId1 = procExec.submitProcedure(
|
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
// Split region of the table with the same nonce
|
|
||||||
long procId2 = procExec.submitProcedure(
|
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
|
||||||
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);
|
|
||||||
|
|
||||||
verify(tableName, splitRowNum);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testInvalidSplitKey() throws Exception {
|
public void testInvalidSplitKey() throws Exception {
|
||||||
final TableName tableName = TableName.valueOf("testInvalidSplitKey");
|
final TableName tableName = TableName.valueOf("testInvalidSplitKey");
|
||||||
|
@ -317,9 +265,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
// Split region of the table with null split key
|
// Split region of the table with null split key
|
||||||
try {
|
try {
|
||||||
long procId1 = procExec.submitProcedure(
|
long procId1 = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], null),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], null));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
ProcedureTestingUtility.waitProcedure(procExec, procId1);
|
||||||
fail("unexpected procedure start with invalid split-key");
|
fail("unexpected procedure start with invalid split-key");
|
||||||
} catch (DoNotRetryIOException e) {
|
} catch (DoNotRetryIOException e) {
|
||||||
|
@ -345,9 +291,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
|
// Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
|
||||||
// rollback
|
// rollback
|
||||||
|
@ -378,9 +322,7 @@ public class TestSplitTableRegionProcedure {
|
||||||
|
|
||||||
// Split region of the table
|
// Split region of the table
|
||||||
long procId = procExec.submitProcedure(
|
long procId = procExec.submitProcedure(
|
||||||
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey),
|
new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
|
||||||
nonceGroup,
|
|
||||||
nonce);
|
|
||||||
|
|
||||||
// Restart the executor and execute the step twice
|
// Restart the executor and execute the step twice
|
||||||
int numberOfSteps = SplitTableRegionState.values().length;
|
int numberOfSteps = SplitTableRegionState.values().length;
|
||||||
|
|
|
@ -164,9 +164,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
|
||||||
|
|
||||||
// Start the Truncate procedure && kill the executor
|
// Start the Truncate procedure && kill the executor
|
||||||
long procId = procExec.submitProcedure(
|
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
|
// Restart the executor and execute the step twice
|
||||||
// NOTE: the 7 (number of TruncateTableState steps) is hardcoded,
|
// NOTE: the 7 (number of TruncateTableState steps) is hardcoded,
|
||||||
|
|
|
@ -20,36 +20,33 @@ package org.apache.hadoop.hbase.security.access;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.client.Admin;
|
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
|
||||||
import org.apache.hadoop.hbase.Coprocessor;
|
import org.apache.hadoop.hbase.Coprocessor;
|
||||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SecurityTests;
|
import org.apache.hadoop.hbase.testclassification.SecurityTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
|
||||||
import org.junit.rules.TestRule;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestRule;
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs coprocessor loads for variuos paths and malformed strings
|
* Performs coprocessor loads for variuos paths and malformed strings
|
||||||
|
@ -58,7 +55,6 @@ import java.io.IOException;
|
||||||
public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
private static final Log LOG = LogFactory.getLog(TestCoprocessorWhitelistMasterObserver.class);
|
private static final Log LOG = LogFactory.getLog(TestCoprocessorWhitelistMasterObserver.class);
|
||||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||||
private static Configuration conf;
|
|
||||||
private static final TableName TEST_TABLE = TableName.valueOf("testTable");
|
private static final TableName TEST_TABLE = TableName.valueOf("testTable");
|
||||||
private static final byte[] TEST_FAMILY = Bytes.toBytes("fam1");
|
private static final byte[] TEST_FAMILY = Bytes.toBytes("fam1");
|
||||||
|
|
||||||
|
@ -106,8 +102,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
// set retries low to raise exception quickly
|
// set retries low to raise exception quickly
|
||||||
conf.setInt("hbase.client.retries.number", 1);
|
conf.setInt("hbase.client.retries.number", 1);
|
||||||
UTIL.startMiniCluster();
|
UTIL.startMiniCluster();
|
||||||
Table table = UTIL.createTable(TEST_TABLE,
|
UTIL.createTable(TEST_TABLE, new byte[][] { TEST_FAMILY });
|
||||||
new byte[][] { TEST_FAMILY });
|
|
||||||
UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
|
UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
|
||||||
Connection connection = ConnectionFactory.createConnection(conf);
|
Connection connection = ConnectionFactory.createConnection(conf);
|
||||||
Table t = connection.getTable(TEST_TABLE);
|
Table t = connection.getTable(TEST_TABLE);
|
||||||
|
@ -139,18 +134,17 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
private static void negativeTestCase(String[] whitelistedPaths,
|
private static void negativeTestCase(String[] whitelistedPaths,
|
||||||
String coprocessorPath) throws Exception {
|
String coprocessorPath) throws Exception {
|
||||||
Configuration conf = UTIL.getConfiguration();
|
Configuration conf = UTIL.getConfiguration();
|
||||||
|
conf.setInt("hbase.client.retries.number", 1);
|
||||||
// load coprocessor under test
|
// load coprocessor under test
|
||||||
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
|
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
|
||||||
CoprocessorWhitelistMasterObserver.class.getName());
|
CoprocessorWhitelistMasterObserver.class.getName());
|
||||||
// set retries low to raise exception quickly
|
// set retries low to raise exception quickly
|
||||||
conf.setInt("hbase.client.retries.number", 1);
|
|
||||||
// set a coprocessor whitelist path for test
|
// set a coprocessor whitelist path for test
|
||||||
conf.setStrings(
|
conf.setStrings(
|
||||||
CoprocessorWhitelistMasterObserver.CP_COPROCESSOR_WHITELIST_PATHS_KEY,
|
CoprocessorWhitelistMasterObserver.CP_COPROCESSOR_WHITELIST_PATHS_KEY,
|
||||||
whitelistedPaths);
|
whitelistedPaths);
|
||||||
UTIL.startMiniCluster();
|
UTIL.startMiniCluster();
|
||||||
Table table = UTIL.createTable(TEST_TABLE,
|
UTIL.createTable(TEST_TABLE, new byte[][] { TEST_FAMILY });
|
||||||
new byte[][] { TEST_FAMILY });
|
|
||||||
UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
|
UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
|
||||||
Connection connection = ConnectionFactory.createConnection(conf);
|
Connection connection = ConnectionFactory.createConnection(conf);
|
||||||
Admin admin = connection.getAdmin();
|
Admin admin = connection.getAdmin();
|
||||||
|
@ -175,7 +169,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* to show coprocessor is working as desired
|
* to show coprocessor is working as desired
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testSubstringNonWhitelisted() throws Exception {
|
public void testSubstringNonWhitelisted() throws Exception {
|
||||||
positiveTestCase(new String[]{"/permitted/*"},
|
positiveTestCase(new String[]{"/permitted/*"},
|
||||||
"file:///notpermitted/couldnotpossiblyexist.jar");
|
"file:///notpermitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -189,7 +182,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testDifferentFileSystemNonWhitelisted() throws Exception {
|
public void testDifferentFileSystemNonWhitelisted() throws Exception {
|
||||||
positiveTestCase(new String[]{"hdfs://foo/bar"},
|
positiveTestCase(new String[]{"hdfs://foo/bar"},
|
||||||
"file:///notpermitted/couldnotpossiblyexist.jar");
|
"file:///notpermitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -203,7 +195,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testSchemeAndDirectorywhitelisted() throws Exception {
|
public void testSchemeAndDirectorywhitelisted() throws Exception {
|
||||||
negativeTestCase(new String[]{"/tmp","file:///permitted/*"},
|
negativeTestCase(new String[]{"/tmp","file:///permitted/*"},
|
||||||
"file:///permitted/couldnotpossiblyexist.jar");
|
"file:///permitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -217,7 +208,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testSchemeWhitelisted() throws Exception {
|
public void testSchemeWhitelisted() throws Exception {
|
||||||
negativeTestCase(new String[]{"file:///"},
|
negativeTestCase(new String[]{"file:///"},
|
||||||
"file:///permitted/couldnotpossiblyexist.jar");
|
"file:///permitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -231,7 +221,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testDFSNameWhitelistedWorks() throws Exception {
|
public void testDFSNameWhitelistedWorks() throws Exception {
|
||||||
negativeTestCase(new String[]{"hdfs://Your-FileSystem"},
|
negativeTestCase(new String[]{"hdfs://Your-FileSystem"},
|
||||||
"hdfs://Your-FileSystem/permitted/couldnotpossiblyexist.jar");
|
"hdfs://Your-FileSystem/permitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -245,7 +234,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testDFSNameNotWhitelistedFails() throws Exception {
|
public void testDFSNameNotWhitelistedFails() throws Exception {
|
||||||
positiveTestCase(new String[]{"hdfs://Your-FileSystem"},
|
positiveTestCase(new String[]{"hdfs://Your-FileSystem"},
|
||||||
"hdfs://My-FileSystem/permitted/couldnotpossiblyexist.jar");
|
"hdfs://My-FileSystem/permitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -259,7 +247,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* the added coprocessor not actually existing on disk
|
* the added coprocessor not actually existing on disk
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testBlanketWhitelist() throws Exception {
|
public void testBlanketWhitelist() throws Exception {
|
||||||
negativeTestCase(new String[]{"*"},
|
negativeTestCase(new String[]{"*"},
|
||||||
"hdfs:///permitted/couldnotpossiblyexist.jar");
|
"hdfs:///permitted/couldnotpossiblyexist.jar");
|
||||||
|
@ -271,7 +258,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* @result Table will not be created due to the offending coprocessor
|
* @result Table will not be created due to the offending coprocessor
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testCreationNonWhitelistedCoprocessorPath() throws Exception {
|
public void testCreationNonWhitelistedCoprocessorPath() throws Exception {
|
||||||
Configuration conf = UTIL.getConfiguration();
|
Configuration conf = UTIL.getConfiguration();
|
||||||
// load coprocessor under test
|
// load coprocessor under test
|
||||||
|
@ -309,7 +295,6 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
|
||||||
* @result Table will be created with the coprocessor
|
* @result Table will be created with the coprocessor
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
@Category(MediumTests.class)
|
|
||||||
public void testCreationClasspathCoprocessor() throws Exception {
|
public void testCreationClasspathCoprocessor() throws Exception {
|
||||||
Configuration conf = UTIL.getConfiguration();
|
Configuration conf = UTIL.getConfiguration();
|
||||||
// load coprocessor under test
|
// load coprocessor under test
|
||||||
|
|
Loading…
Reference in New Issue