Browse Source

HDFS-8950. NameNode refresh doesn't remove DataNodes that are no longer in the allowed list (Daniel Templeton)

(cherry picked from commit b94b56806d3d6e04984e229b479f7ac15b62bbfa)
Colin Patrick Mccabe 9 years ago
parent
commit
159969f658

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -907,6 +907,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8963. Fix incorrect sign extension of xattr length in HDFS-8900.
     (Colin Patrick McCabe via yliu)
 
+    HDFS-8950. NameNode refresh doesn't remove DataNodes that are no longer in
+    the allowed list (Daniel Templeton)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -1278,11 +1278,14 @@ public class DatanodeManager {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
         final boolean isDead = isDatanodeDead(dn);
         final boolean isDecommissioning = dn.isDecommissionInProgress();
-        if ((listLiveNodes && !isDead) ||
+
+        if (((listLiveNodes && !isDead) ||
             (listDeadNodes && isDead) ||
-            (listDecommissioningNodes && isDecommissioning)) {
-            nodes.add(dn);
+            (listDecommissioningNodes && isDecommissioning)) &&
+            hostFileManager.isIncluded(dn)) {
+          nodes.add(dn);
         }
+
         foundNodes.add(HostFileManager.resolvedAddressFromDatanodeID(dn));
       }
     }

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java

@@ -126,9 +126,28 @@ class HostFileManager {
     return !includes.isEmpty();
   }
 
+  /**
+   * Read the includes and excludes lists from the named files.  Any previous
+   * includes and excludes lists are discarded.
+   * @param includeFile the path to the new includes list
+   * @param excludeFile the path to the new excludes list
+   * @throws IOException thrown if there is a problem reading one of the files
+   */
   void refresh(String includeFile, String excludeFile) throws IOException {
     HostSet newIncludes = readFile("included", includeFile);
     HostSet newExcludes = readFile("excluded", excludeFile);
+
+    refresh(newIncludes, newExcludes);
+  }
+
+  /**
+   * Set the includes and excludes lists by the new HostSet instances. The
+   * old instances are discarded.
+   * @param newIncludes the new includes list
+   * @param newExcludes the new excludes list
+   */
+  @VisibleForTesting
+  void refresh(HostSet newIncludes, HostSet newExcludes) {
     synchronized (this) {
       includes = newIncludes;
       excludes = newExcludes;

+ 6 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -813,16 +813,13 @@ public class TestDecommission {
       }
       assertEquals("Number of live nodes should be 0", 0, info.length);
       
-      // Test that non-live and bogus hostnames are considered "dead".
-      // The dead report should have an entry for (1) the DN  that is
-      // now considered dead because it is no longer allowed to connect
-      // and (2) the bogus entry in the hosts file (these entries are
-      // always added last)
+      // Test that bogus hostnames are considered "dead".
+      // The dead report should have an entry for the bogus entry in the hosts
+      // file.  The original datanode is excluded from the report because it
+      // is no longer in the included list.
       info = client.datanodeReport(DatanodeReportType.DEAD);
-      assertEquals("There should be 2 dead nodes", 2, info.length);
-      DatanodeID id = cluster.getDataNodes().get(0).getDatanodeId();
-      assertEquals(id.getHostName(), info[0].getHostName());
-      assertEquals(bogusIp, info[1].getHostName());
+      assertEquals("There should be 1 dead node", 1, info.length);
+      assertEquals(bogusIp, info[0].getHostName());
     }
   }
   

+ 102 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java

@@ -19,11 +19,13 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -38,17 +40,23 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.util.Shell;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.*;
 
@@ -68,6 +76,15 @@ public class TestDatanodeManager {
     return dm;
   }
 
+  /**
+   * Create an InetSocketAddress for a host:port string
+   * @param host a host identifier in host:port format
+   * @return a corresponding InetSocketAddress object
+   */
+  private static InetSocketAddress entry(String host) {
+    return HostFileManager.parseEntry("dummy", "dummy", host);
+  }
+
   /**
    * This test sends a random sequence of node registrations and node removals
    * to the DatanodeManager (of nodes with different IDs and versions), and
@@ -352,5 +369,89 @@ public class TestDatanodeManager {
     assertThat(sortedLocs[sortedLocs.length - 2].getAdminState(),
       is(DatanodeInfo.AdminStates.DECOMMISSIONED));
   }
-}
 
+  /**
+   * Test whether removing a host from the includes list without adding it to
+   * the excludes list will exclude it from data node reports.
+   */
+  @Test
+  public void testRemoveIncludedNode() throws IOException {
+    FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
+
+    // Set the write lock so that the DatanodeManager can start
+    Mockito.when(fsn.hasWriteLock()).thenReturn(true);
+
+    DatanodeManager dm = mockDatanodeManager(fsn, new Configuration());
+    HostFileManager hm = new HostFileManager();
+    HostFileManager.HostSet noNodes = new HostFileManager.HostSet();
+    HostFileManager.HostSet oneNode = new HostFileManager.HostSet();
+    HostFileManager.HostSet twoNodes = new HostFileManager.HostSet();
+    DatanodeRegistration dr1 = new DatanodeRegistration(
+      new DatanodeID("127.0.0.1", "127.0.0.1", "someStorageID-123",
+          12345, 12345, 12345, 12345),
+      new StorageInfo(HdfsServerConstants.NodeType.DATA_NODE),
+      new ExportedBlockKeys(), "test");
+    DatanodeRegistration dr2 = new DatanodeRegistration(
+      new DatanodeID("127.0.0.1", "127.0.0.1", "someStorageID-234",
+          23456, 23456, 23456, 23456),
+      new StorageInfo(HdfsServerConstants.NodeType.DATA_NODE),
+      new ExportedBlockKeys(), "test");
+
+    twoNodes.add(entry("127.0.0.1:12345"));
+    twoNodes.add(entry("127.0.0.1:23456"));
+    oneNode.add(entry("127.0.0.1:23456"));
+
+    hm.refresh(twoNodes, noNodes);
+    Whitebox.setInternalState(dm, "hostFileManager", hm);
+
+    // Register two data nodes to simulate them coming up.
+    // We need to add two nodes, because if we have only one node, removing it
+    // will cause the includes list to be empty, which means all hosts will be
+    // allowed.
+    dm.registerDatanode(dr1);
+    dm.registerDatanode(dr2);
+
+    // Make sure that both nodes are reported
+    List<DatanodeDescriptor> both =
+        dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL);
+
+    // Sort the list so that we know which one is which
+    Collections.sort(both);
+
+    Assert.assertEquals("Incorrect number of hosts reported",
+        2, both.size());
+    Assert.assertEquals("Unexpected host or host in unexpected position",
+        "127.0.0.1:12345", both.get(0).getInfoAddr());
+    Assert.assertEquals("Unexpected host or host in unexpected position",
+        "127.0.0.1:23456", both.get(1).getInfoAddr());
+
+    // Remove one node from includes, but do not add it to excludes.
+    hm.refresh(oneNode, noNodes);
+
+    // Make sure that only one node is still reported
+    List<DatanodeDescriptor> onlyOne =
+        dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL);
+
+    Assert.assertEquals("Incorrect number of hosts reported",
+        1, onlyOne.size());
+    Assert.assertEquals("Unexpected host reported",
+        "127.0.0.1:23456", onlyOne.get(0).getInfoAddr());
+
+    // Remove all nodes from includes
+    hm.refresh(noNodes, noNodes);
+
+    // Check that both nodes are reported again
+    List<DatanodeDescriptor> bothAgain =
+        dm.getDatanodeListForReport(HdfsConstants.DatanodeReportType.ALL);
+
+    // Sort the list so that we know which one is which
+    Collections.sort(bothAgain);
+
+    Assert.assertEquals("Incorrect number of hosts reported",
+        2, bothAgain.size());
+    Assert.assertEquals("Unexpected host or host in unexpected position",
+        "127.0.0.1:12345", bothAgain.get(0).getInfoAddr());
+    Assert.assertEquals("Unexpected host or host in unexpected position",
+        "127.0.0.1:23456", bothAgain.get(1).getInfoAddr());
+  }
+}

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java

@@ -104,23 +104,22 @@ public class TestHostFileManager {
     BlockManager bm = mock(BlockManager.class);
     FSNamesystem fsn = mock(FSNamesystem.class);
     Configuration conf = new Configuration();
-    HostFileManager hm = mock(HostFileManager.class);
+    HostFileManager hm = new HostFileManager();
     HostFileManager.HostSet includedNodes = new HostFileManager.HostSet();
     HostFileManager.HostSet excludedNodes = new HostFileManager.HostSet();
 
     includedNodes.add(entry("127.0.0.1:12345"));
     includedNodes.add(entry("localhost:12345"));
     includedNodes.add(entry("127.0.0.1:12345"));
-
     includedNodes.add(entry("127.0.0.2"));
+
     excludedNodes.add(entry("127.0.0.1:12346"));
     excludedNodes.add(entry("127.0.30.1:12346"));
 
     Assert.assertEquals(2, includedNodes.size());
     Assert.assertEquals(2, excludedNodes.size());
 
-    doReturn(includedNodes).when(hm).getIncludes();
-    doReturn(excludedNodes).when(hm).getExcludes();
+    hm.refresh(includedNodes, excludedNodes);
 
     DatanodeManager dm = new DatanodeManager(bm, fsn, conf);
     Whitebox.setInternalState(dm, "hostFileManager", hm);