|
@@ -19,21 +19,14 @@
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
-import org.apache.hadoop.hdfs.DFSClient;
|
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY;
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
import static org.mockito.ArgumentMatchers.anyList;
|
|
import static org.mockito.ArgumentMatchers.anyList;
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
import static org.mockito.ArgumentMatchers.anyBoolean;
|
|
import static org.mockito.ArgumentMatchers.anyBoolean;
|
|
import static org.mockito.ArgumentMatchers.anyLong;
|
|
import static org.mockito.ArgumentMatchers.anyLong;
|
|
-import static org.mockito.ArgumentMatchers.anyString;
|
|
|
|
-import static org.mockito.Mockito.doAnswer;
|
|
|
|
import static org.mockito.Mockito.doReturn;
|
|
import static org.mockito.Mockito.doReturn;
|
|
import static org.mockito.Mockito.doThrow;
|
|
import static org.mockito.Mockito.doThrow;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -44,16 +37,13 @@ import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.io.OutputStream;
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URISyntaxException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
-import java.util.Arrays;
|
|
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
-import java.util.Random;
|
|
|
|
import java.util.concurrent.Semaphore;
|
|
import java.util.concurrent.Semaphore;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
@@ -78,13 +68,11 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
-import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
@@ -99,11 +87,9 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
|
|
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
|
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
|
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
-import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
|
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.slf4j.event.Level;
|
|
import org.slf4j.event.Level;
|
|
@@ -298,13 +284,13 @@ public class TestBlockRecovery {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /** Sync two replicas */
|
|
|
|
- private void testSyncReplicas(ReplicaRecoveryInfo replica1,
|
|
|
|
- ReplicaRecoveryInfo replica2,
|
|
|
|
- InterDatanodeProtocol dn1,
|
|
|
|
- InterDatanodeProtocol dn2,
|
|
|
|
- long expectLen) throws IOException {
|
|
|
|
-
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Sync two replicas.
|
|
|
|
+ */
|
|
|
|
+ private void testSyncReplicas(ReplicaRecoveryInfo replica1,
|
|
|
|
+ ReplicaRecoveryInfo replica2, InterDatanodeProtocol dn1,
|
|
|
|
+ InterDatanodeProtocol dn2) throws IOException {
|
|
|
|
+
|
|
DatanodeInfo[] locs = new DatanodeInfo[]{
|
|
DatanodeInfo[] locs = new DatanodeInfo[]{
|
|
mock(DatanodeInfo.class), mock(DatanodeInfo.class)};
|
|
mock(DatanodeInfo.class), mock(DatanodeInfo.class)};
|
|
RecoveringBlock rBlock = new RecoveringBlock(block, locs, RECOVERY_ID);
|
|
RecoveringBlock rBlock = new RecoveringBlock(block, locs, RECOVERY_ID);
|
|
@@ -315,7 +301,7 @@ public class TestBlockRecovery {
|
|
DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234), dn2, replica2);
|
|
DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234), dn2, replica2);
|
|
syncList.add(record1);
|
|
syncList.add(record1);
|
|
syncList.add(record2);
|
|
syncList.add(record2);
|
|
-
|
|
|
|
|
|
+
|
|
when(dn1.updateReplicaUnderRecovery(any(ExtendedBlock.class), anyLong(),
|
|
when(dn1.updateReplicaUnderRecovery(any(ExtendedBlock.class), anyLong(),
|
|
anyLong(), anyLong())).thenReturn("storage1");
|
|
anyLong(), anyLong())).thenReturn("storage1");
|
|
when(dn2.updateReplicaUnderRecovery(any(ExtendedBlock.class), anyLong(),
|
|
when(dn2.updateReplicaUnderRecovery(any(ExtendedBlock.class), anyLong(),
|
|
@@ -325,7 +311,7 @@ public class TestBlockRecovery {
|
|
recoveryWorker.new RecoveryTaskContiguous(rBlock);
|
|
recoveryWorker.new RecoveryTaskContiguous(rBlock);
|
|
RecoveryTaskContiguous.syncBlock(syncList);
|
|
RecoveryTaskContiguous.syncBlock(syncList);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* BlockRecovery_02.8.
|
|
* BlockRecovery_02.8.
|
|
* Two replicas are in Finalized state
|
|
* Two replicas are in Finalized state
|
|
@@ -336,38 +322,38 @@ public class TestBlockRecovery {
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-2, ReplicaState.FINALIZED);
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.FINALIZED);
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 2, ReplicaState.FINALIZED);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
|
|
|
|
// two finalized replicas have different length
|
|
// two finalized replicas have different length
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN2, GEN_STAMP-2, ReplicaState.FINALIZED);
|
|
|
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.FINALIZED);
|
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN2, GEN_STAMP - 2, ReplicaState.FINALIZED);
|
|
|
|
|
|
try {
|
|
try {
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
Assert.fail("Two finalized replicas should not have different lengthes!");
|
|
Assert.fail("Two finalized replicas should not have different lengthes!");
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
Assert.assertTrue(e.getMessage().startsWith(
|
|
Assert.assertTrue(e.getMessage().startsWith(
|
|
"Inconsistent size of finalized replicas. "));
|
|
"Inconsistent size of finalized replicas. "));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* BlockRecovery_02.9.
|
|
* BlockRecovery_02.9.
|
|
- * One replica is Finalized and another is RBW.
|
|
|
|
|
|
+ * One replica is Finalized and another is RBW.
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
@Test(timeout=60000)
|
|
@Test(timeout=60000)
|
|
@@ -375,80 +361,81 @@ public class TestBlockRecovery {
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
// rbw and finalized replicas have the same length
|
|
// rbw and finalized replicas have the same length
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RBW);
|
|
REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
-
|
|
|
|
|
|
+
|
|
// rbw replica has a different length from the finalized one
|
|
// rbw replica has a different length from the finalized one
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.FINALIZED);
|
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN2, GEN_STAMP - 2, ReplicaState.RBW);
|
|
|
|
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
dn2 = mock(InterDatanodeProtocol.class);
|
|
dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* BlockRecovery_02.10.
|
|
* BlockRecovery_02.10.
|
|
- * One replica is Finalized and another is RWR.
|
|
|
|
|
|
+ * One replica is Finalized and another is RWR.
|
|
|
|
+ *
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
- @Test(timeout=60000)
|
|
|
|
|
|
+ @Test(timeout = 60000)
|
|
public void testFinalizedRwrReplicas() throws IOException {
|
|
public void testFinalizedRwrReplicas() throws IOException {
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
// rbw and finalized replicas have the same length
|
|
// rbw and finalized replicas have the same length
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.FINALIZED);
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
-
|
|
|
|
|
|
+
|
|
// rbw replica has a different length from the finalized one
|
|
// rbw replica has a different length from the finalized one
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.FINALIZED);
|
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN2, GEN_STAMP - 2, ReplicaState.RBW);
|
|
|
|
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
dn2 = mock(InterDatanodeProtocol.class);
|
|
dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
REPLICA_LEN1);
|
|
REPLICA_LEN1);
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* BlockRecovery_02.11.
|
|
* BlockRecovery_02.11.
|
|
* Two replicas are RBW.
|
|
* Two replicas are RBW.
|
|
@@ -456,26 +443,27 @@ public class TestBlockRecovery {
|
|
*/
|
|
*/
|
|
@Test(timeout=60000)
|
|
@Test(timeout=60000)
|
|
public void testRBWReplicas() throws IOException {
|
|
public void testRBWReplicas() throws IOException {
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW);
|
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.RBW);
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN2, GEN_STAMP - 2, ReplicaState.RBW);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
|
|
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, minLen);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* BlockRecovery_02.12.
|
|
* BlockRecovery_02.12.
|
|
- * One replica is RBW and another is RWR.
|
|
|
|
|
|
+ * One replica is RBW and another is RWR.
|
|
|
|
+ *
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
@Test(timeout=60000)
|
|
@Test(timeout=60000)
|
|
@@ -483,44 +471,45 @@ public class TestBlockRecovery {
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW);
|
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.RBW);
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
- * BlockRecovery_02.13.
|
|
|
|
|
|
+ * BlockRecovery_02.13.
|
|
* Two replicas are RWR.
|
|
* Two replicas are RWR.
|
|
|
|
+ *
|
|
* @throws IOException in case of an error
|
|
* @throws IOException in case of an error
|
|
*/
|
|
*/
|
|
@Test(timeout=60000)
|
|
@Test(timeout=60000)
|
|
public void testRWRReplicas() throws IOException {
|
|
public void testRWRReplicas() throws IOException {
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
}
|
|
}
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RWR);
|
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
- REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN1, GEN_STAMP - 1, ReplicaState.RWR);
|
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
|
+ REPLICA_LEN2, GEN_STAMP - 2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
|
|
|
|
|
|
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
|
|
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
|
|
- testSyncReplicas(replica1, replica2, dn1, dn2, minLen);
|
|
|
|
-
|
|
|
|
|
|
+ testSyncReplicas(replica1, replica2, dn1, dn2);
|
|
|
|
+
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
- }
|
|
|
|
|
|
+ }
|
|
|
|
|
|
private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
|
|
private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
|
|
Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
|
|
Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
|
|
@@ -708,132 +697,6 @@ public class TestBlockRecovery {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- @Test(timeout = 60000)
|
|
|
|
- public void testEcRecoverBlocks() throws Throwable {
|
|
|
|
- // Stop the Mocked DN started in startup()
|
|
|
|
- tearDown();
|
|
|
|
- ErasureCodingPolicy ecPolicy = StripedFileTestUtil.getDefaultECPolicy();
|
|
|
|
- MiniDFSCluster cluster =
|
|
|
|
- new MiniDFSCluster.Builder(conf).numDataNodes(8).build();
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- cluster.waitActive();
|
|
|
|
- NamenodeProtocols preSpyNN = cluster.getNameNodeRpc();
|
|
|
|
- NamenodeProtocols spyNN = spy(preSpyNN);
|
|
|
|
-
|
|
|
|
- // Delay completeFile
|
|
|
|
- GenericTestUtils.DelayAnswer delayer =
|
|
|
|
- new GenericTestUtils.DelayAnswer(LOG);
|
|
|
|
- doAnswer(delayer).when(spyNN).complete(anyString(), anyString(), any(),
|
|
|
|
- anyLong());
|
|
|
|
- String topDir = "/myDir";
|
|
|
|
- DFSClient client = new DFSClient(null, spyNN, conf, null);
|
|
|
|
- Path file = new Path(topDir + "/testECLeaseRecover");
|
|
|
|
- client.mkdirs(topDir, null, false);
|
|
|
|
- client.enableErasureCodingPolicy(ecPolicy.getName());
|
|
|
|
- client.setErasureCodingPolicy(topDir, ecPolicy.getName());
|
|
|
|
- OutputStream stm = client.create(file.toString(), true);
|
|
|
|
-
|
|
|
|
- // write 5MB File
|
|
|
|
- AppendTestUtil.write(stm, 0, 1024 * 1024 * 5);
|
|
|
|
- final AtomicReference<Throwable> err = new AtomicReference<Throwable>();
|
|
|
|
- Thread t = new Thread() {
|
|
|
|
- @Override
|
|
|
|
- public void run() {
|
|
|
|
- try {
|
|
|
|
- stm.close();
|
|
|
|
- } catch (Throwable t) {
|
|
|
|
- err.set(t);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
- t.start();
|
|
|
|
-
|
|
|
|
- // Waiting for close to get to latch
|
|
|
|
- delayer.waitForCall();
|
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
- @Override
|
|
|
|
- public Boolean get() {
|
|
|
|
- try {
|
|
|
|
- return client.getNamenode().recoverLease(file.toString(),
|
|
|
|
- client.getClientName());
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }, 5000, 24000);
|
|
|
|
- delayer.proceed();
|
|
|
|
- } finally {
|
|
|
|
- cluster.shutdown();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Test to verify the race between finalizeBlock and Lease recovery
|
|
|
|
- *
|
|
|
|
- * @throws Exception
|
|
|
|
- */
|
|
|
|
- @Test(timeout = 20000)
|
|
|
|
- public void testRaceBetweenReplicaRecoveryAndFinalizeBlock() throws Exception {
|
|
|
|
- tearDown();// Stop the Mocked DN started in startup()
|
|
|
|
-
|
|
|
|
- Configuration conf = new HdfsConfiguration();
|
|
|
|
- conf.set(DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY, "1000");
|
|
|
|
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
- .numDataNodes(1).build();
|
|
|
|
- try {
|
|
|
|
- cluster.waitClusterUp();
|
|
|
|
- DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
|
- Path path = new Path("/test");
|
|
|
|
- FSDataOutputStream out = fs.create(path);
|
|
|
|
- out.writeBytes("data");
|
|
|
|
- out.hsync();
|
|
|
|
-
|
|
|
|
- List<LocatedBlock> blocks = DFSTestUtil.getAllBlocks(fs.open(path));
|
|
|
|
- final LocatedBlock block = blocks.get(0);
|
|
|
|
- final DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
-
|
|
|
|
- final AtomicBoolean recoveryInitResult = new AtomicBoolean(true);
|
|
|
|
- Thread recoveryThread = new Thread() {
|
|
|
|
- @Override
|
|
|
|
- public void run() {
|
|
|
|
- try {
|
|
|
|
- DatanodeInfo[] locations = block.getLocations();
|
|
|
|
- final RecoveringBlock recoveringBlock = new RecoveringBlock(
|
|
|
|
- block.getBlock(), locations, block.getBlock()
|
|
|
|
- .getGenerationStamp() + 1);
|
|
|
|
- try(AutoCloseableLock lock = dataNode.data.acquireDatasetLock()) {
|
|
|
|
- Thread.sleep(2000);
|
|
|
|
- dataNode.initReplicaRecovery(recoveringBlock);
|
|
|
|
- }
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- recoveryInitResult.set(false);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
- recoveryThread.start();
|
|
|
|
- try {
|
|
|
|
- out.close();
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- Assert.assertTrue("Writing should fail",
|
|
|
|
- e.getMessage().contains("are bad. Aborting..."));
|
|
|
|
- } finally {
|
|
|
|
- recoveryThread.join();
|
|
|
|
- }
|
|
|
|
- Assert.assertTrue("Recovery should be initiated successfully",
|
|
|
|
- recoveryInitResult.get());
|
|
|
|
-
|
|
|
|
- dataNode.updateReplicaUnderRecovery(block.getBlock(), block.getBlock()
|
|
|
|
- .getGenerationStamp() + 1, block.getBlock().getBlockId(),
|
|
|
|
- block.getBlockSize());
|
|
|
|
- } finally {
|
|
|
|
- if (null != cluster) {
|
|
|
|
- cluster.shutdown();
|
|
|
|
- cluster = null;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* DNs report RUR instead of RBW, RWR or FINALIZED. Primary DN expected to
|
|
* DNs report RUR instead of RBW, RWR or FINALIZED. Primary DN expected to
|
|
* throw an exception.
|
|
* throw an exception.
|
|
@@ -1107,57 +970,7 @@ public class TestBlockRecovery {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Test for block recovery taking longer than the heartbeat interval.
|
|
|
|
- */
|
|
|
|
- @Test(timeout = 300000L)
|
|
|
|
- public void testRecoverySlowerThanHeartbeat() throws Exception {
|
|
|
|
- tearDown(); // Stop the Mocked DN started in startup()
|
|
|
|
-
|
|
|
|
- SleepAnswer delayer = new SleepAnswer(3000, 6000);
|
|
|
|
- testRecoveryWithDatanodeDelayed(delayer);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Test for block recovery timeout. All recovery attempts will be delayed
|
|
|
|
- * and the first attempt will be lost to trigger recovery timeout and retry.
|
|
|
|
- */
|
|
|
|
- @Test(timeout = 300000L)
|
|
|
|
- public void testRecoveryTimeout() throws Exception {
|
|
|
|
- tearDown(); // Stop the Mocked DN started in startup()
|
|
|
|
- final Random r = new Random();
|
|
|
|
-
|
|
|
|
- // Make sure first commitBlockSynchronization call from the DN gets lost
|
|
|
|
- // for the recovery timeout to expire and new recovery attempt
|
|
|
|
- // to be started.
|
|
|
|
- SleepAnswer delayer = new SleepAnswer(3000) {
|
|
|
|
- private final AtomicBoolean callRealMethod = new AtomicBoolean();
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public Object answer(InvocationOnMock invocation) throws Throwable {
|
|
|
|
- boolean interrupted = false;
|
|
|
|
- try {
|
|
|
|
- Thread.sleep(r.nextInt(3000) + 6000);
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
- interrupted = true;
|
|
|
|
- }
|
|
|
|
- try {
|
|
|
|
- if (callRealMethod.get()) {
|
|
|
|
- return invocation.callRealMethod();
|
|
|
|
- }
|
|
|
|
- callRealMethod.set(true);
|
|
|
|
- return null;
|
|
|
|
- } finally {
|
|
|
|
- if (interrupted) {
|
|
|
|
- Thread.currentThread().interrupt();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
- testRecoveryWithDatanodeDelayed(delayer);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void testRecoveryWithDatanodeDelayed(
|
|
|
|
|
|
+ static void testRecoveryWithDatanodeDelayed(
|
|
GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
|
|
GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
|
|
Configuration configuration = new HdfsConfiguration();
|
|
Configuration configuration = new HdfsConfiguration();
|
|
configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
@@ -1209,80 +1022,4 @@ public class TestBlockRecovery {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Test that block will be recovered even if there are less than the
|
|
|
|
- * specified minReplication datanodes involved in its recovery.
|
|
|
|
- *
|
|
|
|
- * Check that, after recovering, the block will be successfully replicated.
|
|
|
|
- */
|
|
|
|
- @Test(timeout = 300000L)
|
|
|
|
- public void testRecoveryWillIgnoreMinReplication() throws Exception {
|
|
|
|
- tearDown(); // Stop the Mocked DN started in startup()
|
|
|
|
-
|
|
|
|
- final int blockSize = 4096;
|
|
|
|
- final int numReplicas = 3;
|
|
|
|
- final String filename = "/testIgnoreMinReplication";
|
|
|
|
- final Path filePath = new Path(filename);
|
|
|
|
- Configuration configuration = new HdfsConfiguration();
|
|
|
|
- configuration.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
|
|
|
- configuration.setInt(DFS_NAMENODE_REPLICATION_MIN_KEY, 2);
|
|
|
|
- configuration.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
|
- MiniDFSCluster cluster = null;
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- cluster = new MiniDFSCluster.Builder(configuration).numDataNodes(5)
|
|
|
|
- .build();
|
|
|
|
- cluster.waitActive();
|
|
|
|
- final DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
|
- final FSNamesystem fsn = cluster.getNamesystem();
|
|
|
|
-
|
|
|
|
- // Create a file and never close the output stream to trigger recovery
|
|
|
|
- FSDataOutputStream out = dfs.create(filePath, (short) numReplicas);
|
|
|
|
- out.write(AppendTestUtil.randomBytes(0, blockSize));
|
|
|
|
- out.hsync();
|
|
|
|
-
|
|
|
|
- DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
|
|
|
|
- cluster.getNameNodePort()), configuration);
|
|
|
|
- LocatedBlock blk = dfsClient.getNamenode().
|
|
|
|
- getBlockLocations(filename, 0, blockSize).
|
|
|
|
- getLastLocatedBlock();
|
|
|
|
-
|
|
|
|
- // Kill 2 out of 3 datanodes so that only 1 alive, thus < minReplication
|
|
|
|
- List<DatanodeInfo> dataNodes = Arrays.asList(blk.getLocations());
|
|
|
|
- assertEquals(dataNodes.size(), numReplicas);
|
|
|
|
- for (DatanodeInfo dataNode : dataNodes.subList(0, numReplicas - 1)) {
|
|
|
|
- cluster.stopDataNode(dataNode.getName());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
- @Override
|
|
|
|
- public Boolean get() {
|
|
|
|
- return fsn.getNumDeadDataNodes() == 2;
|
|
|
|
- }
|
|
|
|
- }, 300, 300000);
|
|
|
|
-
|
|
|
|
- // Make sure hard lease expires to trigger replica recovery
|
|
|
|
- cluster.setLeasePeriod(100L, 100L);
|
|
|
|
-
|
|
|
|
- // Wait for recovery to succeed
|
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
- @Override
|
|
|
|
- public Boolean get() {
|
|
|
|
- try {
|
|
|
|
- return dfs.isFileClosed(filePath);
|
|
|
|
- } catch (IOException e) {}
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- }, 300, 300000);
|
|
|
|
-
|
|
|
|
- // Wait for the block to be replicated
|
|
|
|
- DFSTestUtil.waitForReplication(cluster, DFSTestUtil.getFirstBlock(
|
|
|
|
- dfs, filePath), 1, numReplicas, 0);
|
|
|
|
-
|
|
|
|
- } finally {
|
|
|
|
- if (cluster != null) {
|
|
|
|
- cluster.shutdown();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
}
|
|
}
|