Bläddra i källkod

HDFS-11896. Non-dfsUsed will be doubled on dead node re-registration. Contributed by Brahma Reddy Battula.

(cherry picked from commit c4a85c694fae3f814ab4e7f3c172da1df0e0e353)(cherry picked from commit 9a168ae884f0075dea4930e07e1ced30a0d0fea6)
Brahma Reddy Battula 7 år sedan
förälder
incheckning
205a95c29b

+ 11 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -311,11 +311,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   public void resetBlocks() {
-    setCapacity(0);
-    setRemaining(0);
-    setBlockPoolUsed(0);
-    setDfsUsed(0);
-    setXceiverCount(0);
+    updateStorageStats(this.getStorageReports(), 0L, 0L, 0, 0, null);
     this.invalidateBlocks.clear();
     this.volumeFailures = 0;
     // pendingCached, cached, and pendingUncached are protected by the
@@ -363,6 +359,16 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public void updateHeartbeatState(StorageReport[] reports, long cacheCapacity,
       long cacheUsed, int xceiverCount, int volFailures,
       VolumeFailureSummary volumeFailureSummary) {
+    updateStorageStats(reports, cacheCapacity, cacheUsed, xceiverCount,
+        volFailures, volumeFailureSummary);
+    setLastUpdate(Time.now());
+    setLastUpdateMonotonic(Time.monotonicNow());
+    rollBlocksScheduled(getLastUpdateMonotonic());
+  }
+
+  private void updateStorageStats(StorageReport[] reports, long cacheCapacity,
+      long cacheUsed, int xceiverCount, int volFailures,
+      VolumeFailureSummary volumeFailureSummary) {
     long totalCapacity = 0;
     long totalRemaining = 0;
     long totalBlockPoolUsed = 0;
@@ -413,8 +419,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setCacheCapacity(cacheCapacity);
     setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
-    setLastUpdate(Time.now());
-    setLastUpdateMonotonic(Time.monotonicNow());
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
@@ -430,7 +434,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
       totalDfsUsed += report.getDfsUsed();
       totalNonDfsUsed += report.getNonDfsUsed();
     }
-    rollBlocksScheduled(getLastUpdateMonotonic());
 
     // Update total metrics for the node.
     setCapacity(totalCapacity);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1217,7 +1217,7 @@ public class DataNode extends ReconfigurableBase
 
   // used only for testing
   @VisibleForTesting
-  void setHeartbeatsDisabledForTests(
+  public void setHeartbeatsDisabledForTests(
       boolean heartbeatsDisabledForTests) {
     this.heartbeatsDisabledForTests = heartbeatsDisabledForTests;
   }

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java

@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import com.google.common.base.Supplier;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -34,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -49,6 +52,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
 
@@ -173,4 +177,53 @@ public class TestDeadDatanode {
           .getDatanodeDescriptor().equals(clientNode));
     }
   }
+
+  @Test
+  public void testNonDFSUsedONDeadNodeReReg() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
+        6 * 1000);
+    long CAPACITY = 5000L;
+    long[] capacities = new long[] { 4 * CAPACITY, 4 * CAPACITY };
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+          .simulatedCapacities(capacities).build();
+      long initialCapacity = cluster.getNamesystem(0).getCapacityTotal();
+      assertTrue(initialCapacity > 0);
+      DataNode dn1 = cluster.getDataNodes().get(0);
+      DataNode dn2 = cluster.getDataNodes().get(1);
+      final DatanodeDescriptor dn2Desc = cluster.getNamesystem(0)
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn2.getDatanodeId());
+      dn1.setHeartbeatsDisabledForTests(true);
+      cluster.setDataNodeDead(dn1.getDatanodeId());
+      assertEquals("Capacity shouldn't include DeadNode", dn2Desc.getCapacity(),
+          cluster.getNamesystem(0).getCapacityTotal());
+      assertEquals("NonDFS-used shouldn't include DeadNode",
+          dn2Desc.getNonDfsUsed(),
+          cluster.getNamesystem(0).getNonDfsUsedSpace());
+      // Wait for re-registration and heartbeat
+      dn1.setHeartbeatsDisabledForTests(false);
+      final DatanodeDescriptor dn1Desc = cluster.getNamesystem(0)
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn1.getDatanodeId());
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+
+        @Override public Boolean get() {
+          return dn1Desc.isAlive() && dn1Desc.isHeartbeatedSinceRegistration();
+        }
+      }, 100, 5000);
+      assertEquals("Capacity should be 0 after all DNs dead", initialCapacity,
+          cluster.getNamesystem(0).getCapacityTotal());
+      long nonDfsAfterReg = cluster.getNamesystem(0).getNonDfsUsedSpace();
+      assertEquals("NonDFS should include actual DN NonDFSUsed",
+          dn1Desc.getNonDfsUsed() + dn2Desc.getNonDfsUsed(), nonDfsAfterReg);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }