|
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
|
|
@@ -83,6 +84,7 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -186,7 +188,8 @@ public class TestBlockRecovery {
|
|
|
Mockito.anyInt(),
|
|
|
Mockito.anyInt(),
|
|
|
Mockito.any(VolumeFailureSummary.class),
|
|
|
- Mockito.anyBoolean()))
|
|
|
+ Mockito.anyBoolean(),
|
|
|
+ Mockito.any(SlowPeerReports.class)))
|
|
|
.thenReturn(new HeartbeatResponse(
|
|
|
new DatanodeCommand[0],
|
|
|
new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),
|
|
@@ -252,15 +255,15 @@ public class TestBlockRecovery {
|
|
|
}
|
|
|
|
|
|
/** Sync two replicas */
|
|
|
- private void testSyncReplicas(ReplicaRecoveryInfo replica1,
|
|
|
+ private void testSyncReplicas(ReplicaRecoveryInfo replica1,
|
|
|
ReplicaRecoveryInfo replica2,
|
|
|
InterDatanodeProtocol dn1,
|
|
|
InterDatanodeProtocol dn2,
|
|
|
long expectLen) throws IOException {
|
|
|
-
|
|
|
+
|
|
|
DatanodeInfo[] locs = new DatanodeInfo[]{
|
|
|
mock(DatanodeInfo.class), mock(DatanodeInfo.class)};
|
|
|
- RecoveringBlock rBlock = new RecoveringBlock(block,
|
|
|
+ RecoveringBlock rBlock = new RecoveringBlock(block,
|
|
|
locs, RECOVERY_ID);
|
|
|
ArrayList<BlockRecord> syncList = new ArrayList<BlockRecord>(2);
|
|
|
BlockRecord record1 = new BlockRecord(
|
|
@@ -269,7 +272,7 @@ public class TestBlockRecovery {
|
|
|
DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234), dn2, replica2);
|
|
|
syncList.add(record1);
|
|
|
syncList.add(record2);
|
|
|
-
|
|
|
+
|
|
|
when(dn1.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(),
|
|
|
anyLong(), anyLong())).thenReturn("storage1");
|
|
|
when(dn2.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(),
|
|
@@ -279,7 +282,7 @@ public class TestBlockRecovery {
|
|
|
recoveryWorker.new RecoveryTaskContiguous(rBlock);
|
|
|
RecoveryTaskContiguous.syncBlock(syncList);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* BlockRecovery_02.8.
|
|
|
* Two replicas are in Finalized state
|
|
@@ -290,9 +293,9 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
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.FINALIZED);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -305,9 +308,9 @@ public class TestBlockRecovery {
|
|
|
REPLICA_LEN1);
|
|
|
|
|
|
// two finalized replicas have different length
|
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN2, GEN_STAMP-2, ReplicaState.FINALIZED);
|
|
|
|
|
|
try {
|
|
@@ -318,10 +321,10 @@ public class TestBlockRecovery {
|
|
|
"Inconsistent size of finalized replicas. "));
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* 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
|
|
|
*/
|
|
|
@Test(timeout=60000)
|
|
@@ -329,11 +332,11 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// 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);
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -344,11 +347,11 @@ public class TestBlockRecovery {
|
|
|
REPLICA_LEN1);
|
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
|
|
|
REPLICA_LEN1);
|
|
|
-
|
|
|
+
|
|
|
// rbw replica has a different length from the finalized one
|
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -359,10 +362,10 @@ public class TestBlockRecovery {
|
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* 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
|
|
|
*/
|
|
|
@Test(timeout=60000)
|
|
@@ -370,11 +373,11 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// 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);
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -385,11 +388,11 @@ public class TestBlockRecovery {
|
|
|
REPLICA_LEN1);
|
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
|
-
|
|
|
+
|
|
|
// rbw replica has a different length from the finalized one
|
|
|
- replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.FINALIZED);
|
|
|
- replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -401,7 +404,7 @@ public class TestBlockRecovery {
|
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* BlockRecovery_02.11.
|
|
|
* Two replicas are RBW.
|
|
@@ -412,9 +415,9 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW);
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RBW);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -425,10 +428,10 @@ public class TestBlockRecovery {
|
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* 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
|
|
|
*/
|
|
|
@Test(timeout=60000)
|
|
@@ -436,9 +439,9 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RBW);
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -450,9 +453,9 @@ public class TestBlockRecovery {
|
|
|
verify(dn2, never()).updateReplicaUnderRecovery(
|
|
|
block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- * BlockRecovery_02.13.
|
|
|
+ * BlockRecovery_02.13.
|
|
|
* Two replicas are RWR.
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
@@ -461,9 +464,9 @@ public class TestBlockRecovery {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Running " + GenericTestUtils.getMethodName());
|
|
|
}
|
|
|
- ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN1, GEN_STAMP-1, ReplicaState.RWR);
|
|
|
- ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
+ ReplicaRecoveryInfo replica2 = new ReplicaRecoveryInfo(BLOCK_ID,
|
|
|
REPLICA_LEN2, GEN_STAMP-2, ReplicaState.RWR);
|
|
|
|
|
|
InterDatanodeProtocol dn1 = mock(InterDatanodeProtocol.class);
|
|
@@ -471,10 +474,10 @@ public class TestBlockRecovery {
|
|
|
|
|
|
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
|
|
|
testSyncReplicas(replica1, replica2, dn1, dn2, minLen);
|
|
|
-
|
|
|
+
|
|
|
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
|
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
|
|
|
- }
|
|
|
+ }
|
|
|
|
|
|
private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
|
|
|
Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
|
|
@@ -661,10 +664,10 @@ public class TestBlockRecovery {
|
|
|
streams.close();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Test to verify the race between finalizeBlock and Lease recovery
|
|
|
- *
|
|
|
+ *
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
@Test(timeout = 20000)
|
|
@@ -682,11 +685,11 @@ public class TestBlockRecovery {
|
|
|
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
|
|
@@ -716,7 +719,7 @@ public class TestBlockRecovery {
|
|
|
}
|
|
|
Assert.assertTrue("Recovery should be initiated successfully",
|
|
|
recoveryInitResult.get());
|
|
|
-
|
|
|
+
|
|
|
dataNode.updateReplicaUnderRecovery(block.getBlock(), block.getBlock()
|
|
|
.getGenerationStamp() + 1, block.getBlock().getBlockId(),
|
|
|
block.getBlockSize());
|