|
@@ -40,10 +40,12 @@ import java.net.URISyntaxException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.Semaphore;
|
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
+import com.google.common.collect.Iterators;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -86,7 +88,9 @@ import org.apache.log4j.Level;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.TestName;
|
|
|
import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -116,12 +120,18 @@ public class TestBlockRecovery {
|
|
|
private final static long REPLICA_LEN2 = 5000L;
|
|
|
private final static ExtendedBlock block = new ExtendedBlock(POOL_ID,
|
|
|
BLOCK_ID, BLOCK_LEN, GEN_STAMP);
|
|
|
-
|
|
|
+
|
|
|
+ @Rule
|
|
|
+ public TestName currentTestName = new TestName();
|
|
|
+
|
|
|
static {
|
|
|
GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
|
|
|
GenericTestUtils.setLogLevel(LOG, Level.ALL);
|
|
|
}
|
|
|
|
|
|
+ private final long
|
|
|
+ TEST_LOCK_HOG_DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS = 1000000000L;
|
|
|
+
|
|
|
/**
|
|
|
* Starts an instance of DataNode
|
|
|
* @throws IOException
|
|
@@ -133,6 +143,12 @@ public class TestBlockRecovery {
|
|
|
conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
|
|
|
+ if (currentTestName.getMethodName().equals(
|
|
|
+ "testInitReplicaRecoveryDoesNotHogLock")) {
|
|
|
+ // This test requires a very long value for the xceiver stop timeout.
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
|
|
|
+ TEST_LOCK_HOG_DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS);
|
|
|
+ }
|
|
|
conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
|
|
|
FileSystem.setDefaultUri(conf,
|
|
|
"hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
|
|
@@ -265,7 +281,7 @@ public class TestBlockRecovery {
|
|
|
* Two replicas are in Finalized state
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testFinalizedReplicas () throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -304,7 +320,7 @@ public class TestBlockRecovery {
|
|
|
* One replica is Finalized and another is RBW.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testFinalizedRbwReplicas() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -345,7 +361,7 @@ public class TestBlockRecovery {
|
|
|
* One replica is Finalized and another is RWR.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testFinalizedRwrReplicas() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -387,7 +403,7 @@ public class TestBlockRecovery {
|
|
|
* Two replicas are RBW.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testRBWReplicas() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -411,7 +427,7 @@ public class TestBlockRecovery {
|
|
|
* One replica is RBW and another is RWR.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testRBW_RWRReplicas() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -436,7 +452,7 @@ public class TestBlockRecovery {
|
|
|
* Two replicas are RWR.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testRWRReplicas() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -472,7 +488,7 @@ public class TestBlockRecovery {
|
|
|
* @throws IOException
|
|
|
* in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testRecoveryInProgressException()
|
|
|
throws IOException, InterruptedException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
@@ -497,7 +513,7 @@ public class TestBlockRecovery {
|
|
|
* @throws IOException
|
|
|
* in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testErrorReplicas() throws IOException, InterruptedException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -524,7 +540,7 @@ public class TestBlockRecovery {
|
|
|
*
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testZeroLenReplicas() throws IOException, InterruptedException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -564,7 +580,7 @@ public class TestBlockRecovery {
|
|
|
*
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testFailedReplicaUpdate() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -586,7 +602,7 @@ public class TestBlockRecovery {
|
|
|
*
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testNoReplicaUnderRecovery() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -611,7 +627,7 @@ public class TestBlockRecovery {
|
|
|
*
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testNotMatchedReplicaID() throws IOException {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -712,7 +728,7 @@ public class TestBlockRecovery {
|
|
|
* throw an exception.
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testRURReplicas() throws Exception {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
@@ -742,4 +758,97 @@ public class TestBlockRecovery {
|
|
|
assertTrue(exceptionThrown);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test that initReplicaRecovery does not hold the lock for an unreasonable
|
|
|
+ * amount of time if a writer is taking a long time to stop.
|
|
|
+ */
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testInitReplicaRecoveryDoesNotHogLock() throws Exception {
|
|
|
+ if(LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
+ }
|
|
|
+ // We need a long value for the data xceiver stop timeout.
|
|
|
+ // Otherwise the timeout will trigger, and we will not have tested that
|
|
|
+ // thread join was done locklessly.
|
|
|
+ Assert.assertEquals(
|
|
|
+ TEST_LOCK_HOG_DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS,
|
|
|
+ dn.getDnConf().getXceiverStopTimeout());
|
|
|
+ final Semaphore progressParent = new Semaphore(0);
|
|
|
+ final Semaphore terminateSlowWorker = new Semaphore(0);
|
|
|
+ final AtomicBoolean failure = new AtomicBoolean(false);
|
|
|
+ Collection<RecoveringBlock> recoveringBlocks =
|
|
|
+ initRecoveringBlocks();
|
|
|
+ final RecoveringBlock recoveringBlock =
|
|
|
+ Iterators.get(recoveringBlocks.iterator(), 0);
|
|
|
+ final ExtendedBlock block = recoveringBlock.getBlock();
|
|
|
+ Thread slowWorker = new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ // Register this thread as the writer for the recoveringBlock.
|
|
|
+ LOG.debug("slowWorker creating rbw");
|
|
|
+ ReplicaHandler replicaHandler =
|
|
|
+ spyDN.data.createRbw(StorageType.DISK, block, false);
|
|
|
+ replicaHandler.close();
|
|
|
+ LOG.debug("slowWorker created rbw");
|
|
|
+ // Tell the parent thread to start progressing.
|
|
|
+ progressParent.release();
|
|
|
+ while (true) {
|
|
|
+ try {
|
|
|
+ terminateSlowWorker.acquire();
|
|
|
+ break;
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Ignore interrupted exceptions so that the waitingWorker thread
|
|
|
+ // will have to wait for us.
|
|
|
+ }
|
|
|
+ }
|
|
|
+ LOG.debug("slowWorker exiting");
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("slowWorker got exception", t);
|
|
|
+ failure.set(true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ // Start the slow worker thread and wait for it to take ownership of the
|
|
|
+ // ReplicaInPipeline
|
|
|
+ slowWorker.start();
|
|
|
+ while (true) {
|
|
|
+ try {
|
|
|
+ progressParent.acquire();
|
|
|
+ break;
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Ignore interrupted exceptions
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Start a worker thread which will wait for the slow worker thread.
|
|
|
+ Thread waitingWorker = new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ // Attempt to terminate the other worker thread and take ownership
|
|
|
+ // of the ReplicaInPipeline.
|
|
|
+ LOG.debug("waitingWorker initiating recovery");
|
|
|
+ spyDN.initReplicaRecovery(recoveringBlock);
|
|
|
+ LOG.debug("waitingWorker initiated recovery");
|
|
|
+ } catch (Throwable t) {
|
|
|
+ GenericTestUtils.assertExceptionContains("meta does not exist", t);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ waitingWorker.start();
|
|
|
+
|
|
|
+ // Do an operation that requires the lock. This should not be blocked
|
|
|
+ // by the replica recovery in progress.
|
|
|
+ spyDN.getFSDataset().getReplicaString(
|
|
|
+ recoveringBlock.getBlock().getBlockPoolId(),
|
|
|
+ recoveringBlock.getBlock().getBlockId());
|
|
|
+
|
|
|
+ // Wait for the two worker threads to exit normally.
|
|
|
+ terminateSlowWorker.release();
|
|
|
+ slowWorker.join();
|
|
|
+ waitingWorker.join();
|
|
|
+ Assert.assertFalse("The slowWriter thread failed.", failure.get());
|
|
|
+ }
|
|
|
}
|