|
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeLifelineProtocolClientSideTranslatorPB;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -90,6 +91,8 @@ public class TestDataNodeLifeline {
|
|
|
private DataNodeMetrics metrics;
|
|
|
private DatanodeProtocolClientSideTranslatorPB namenode;
|
|
|
private FSNamesystem namesystem;
|
|
|
+ private DataNode dn;
|
|
|
+ private BPServiceActor bpsa;
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
@@ -106,7 +109,7 @@ public class TestDataNodeLifeline {
|
|
|
namesystem = cluster.getNameNode().getNamesystem();
|
|
|
|
|
|
// Set up spies on RPC proxies so that we can inject failures.
|
|
|
- DataNode dn = cluster.getDataNodes().get(0);
|
|
|
+ dn = cluster.getDataNodes().get(0);
|
|
|
metrics = dn.getMetrics();
|
|
|
assertNotNull(metrics);
|
|
|
List<BPOfferService> allBpos = dn.getAllBpOs();
|
|
@@ -118,7 +121,7 @@ public class TestDataNodeLifeline {
|
|
|
assertNotNull(allBpsa);
|
|
|
assertEquals(1, allBpsa.size());
|
|
|
|
|
|
- final BPServiceActor bpsa = allBpsa.get(0);
|
|
|
+ bpsa = allBpsa.get(0);
|
|
|
assertNotNull(bpsa);
|
|
|
|
|
|
// Lifeline RPC proxy gets created on separate thread, so poll until found.
|
|
@@ -257,6 +260,32 @@ public class TestDataNodeLifeline {
|
|
|
getLongCounter("LifelinesNumOps", getMetrics(metrics.name())));
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testLifelineForDeadNode() throws Exception {
|
|
|
+ long initialCapacity = cluster.getNamesystem(0).getCapacityTotal();
|
|
|
+ assertTrue(initialCapacity > 0);
|
|
|
+ dn.setHeartbeatsDisabledForTests(true);
|
|
|
+ cluster.setDataNodesDead();
|
|
|
+ assertEquals("Capacity should be 0 after all DNs dead", 0, cluster
|
|
|
+ .getNamesystem(0).getCapacityTotal());
|
|
|
+ bpsa.sendLifelineForTests();
|
|
|
+ assertEquals("Lifeline should be ignored for dead node", 0, cluster
|
|
|
+ .getNamesystem(0).getCapacityTotal());
|
|
|
+ // Wait for re-registration and heartbeat
|
|
|
+ dn.setHeartbeatsDisabledForTests(false);
|
|
|
+ final DatanodeDescriptor dnDesc = cluster.getNamesystem(0).getBlockManager()
|
|
|
+ .getDatanodeManager().getDatanodes().iterator().next();
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ return dnDesc.isAlive() && dnDesc.isHeartbeatedSinceRegistration();
|
|
|
+ }
|
|
|
+ }, 100, 5000);
|
|
|
+ assertEquals("Capacity should include only live capacity", initialCapacity,
|
|
|
+ cluster.getNamesystem(0).getCapacityTotal());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Waits on a {@link CountDownLatch} before calling through to the method.
|
|
|
*/
|