Author: wheat9
Date: Mon Apr  7 23:58:24 2014
New Revision: 1585626

URL: http://svn.apache.org/r1585626
Log:
HDFS-6180. Merge r1585625 from trunk.

Added:
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java
      - copied unchanged from r1585625, 
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
      - copied unchanged from r1585625, 
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
Removed:
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/HostFileManager.java
Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1585626&r1=1585625&r2=1585626&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt 
(original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt 
Mon Apr  7 23:58:24 2014
@@ -66,6 +66,9 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6181. Fix the wrong property names in NFS user guide (brandonli)
 
+    HDFS-6180. dead node count / listing is very broken in JMX and old GUI.
+    (wheat9)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java?rev=1585626&r1=1585625&r2=1585626&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
 Mon Apr  7 23:58:24 2014
@@ -34,10 +34,6 @@ import org.apache.hadoop.hdfs.protocol.H
 import 
org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import 
org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
-import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
-import org.apache.hadoop.hdfs.server.namenode.HostFileManager.Entry;
-import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
-import org.apache.hadoop.hdfs.server.namenode.HostFileManager.MutableEntrySet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.*;
@@ -53,6 +49,7 @@ import org.apache.hadoop.util.Time;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
@@ -211,13 +208,11 @@ public class DatanodeManager {
     // in the cache; so future calls to resolve will be fast.
     if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
       final ArrayList<String> locations = new ArrayList<String>();
-      for (Entry entry : hostFileManager.getIncludes()) {
-        if (!entry.getIpAddress().isEmpty()) {
-          locations.add(entry.getIpAddress());
-        }
+      for (InetSocketAddress addr : hostFileManager.getIncludes()) {
+        locations.add(addr.getAddress().getHostAddress());
       }
       dnsToSwitchMapping.resolve(locations);
-    };
+    }
 
     final long heartbeatIntervalSeconds = conf.getLong(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
@@ -1198,46 +1193,45 @@ public class DatanodeManager {
     boolean listDeadNodes = type == DatanodeReportType.ALL ||
                             type == DatanodeReportType.DEAD;
 
-    ArrayList<DatanodeDescriptor> nodes = null;
-    final MutableEntrySet foundNodes = new MutableEntrySet();
+    ArrayList<DatanodeDescriptor> nodes;
+    final HostFileManager.HostSet foundNodes = new HostFileManager.HostSet();
+    final HostFileManager.HostSet includedNodes = 
hostFileManager.getIncludes();
+    final HostFileManager.HostSet excludedNodes = 
hostFileManager.getExcludes();
+
     synchronized(datanodeMap) {
       nodes = new ArrayList<DatanodeDescriptor>(datanodeMap.size());
-      Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
-      while (it.hasNext()) { 
-        DatanodeDescriptor dn = it.next();
+      for (DatanodeDescriptor dn : datanodeMap.values()) {
         final boolean isDead = isDatanodeDead(dn);
-        if ( (isDead && listDeadNodes) || (!isDead && listLiveNodes) ) {
-          nodes.add(dn);
+        if ((listLiveNodes && !isDead) || (listDeadNodes && isDead)) {
+            nodes.add(dn);
         }
-        foundNodes.add(dn);
+        foundNodes.add(HostFileManager.resolvedAddressFromDatanodeID(dn));
       }
     }
 
     if (listDeadNodes) {
-      final EntrySet includedNodes = hostFileManager.getIncludes();
-      final EntrySet excludedNodes = hostFileManager.getExcludes();
-      for (Entry entry : includedNodes) {
-        if ((foundNodes.find(entry) == null) &&
-            (excludedNodes.find(entry) == null)) {
-          // The remaining nodes are ones that are referenced by the hosts
-          // files but that we do not know about, ie that we have never
-          // head from. Eg. an entry that is no longer part of the cluster
-          // or a bogus entry was given in the hosts files
-          //
-          // If the host file entry specified the xferPort, we use that.
-          // Otherwise, we guess that it is the default xfer port.
-          // We can't ask the DataNode what it had configured, because it's
-          // dead.
-          DatanodeDescriptor dn =
-              new DatanodeDescriptor(new DatanodeID(entry.getIpAddress(),
-                  entry.getPrefix(), "",
-                  entry.getPort() == 0 ? defaultXferPort : entry.getPort(),
-                  defaultInfoPort, defaultInfoSecurePort, defaultIpcPort));
-          dn.setLastUpdate(0); // Consider this node dead for reporting
-          nodes.add(dn);
-        }
+      for (InetSocketAddress addr : includedNodes) {
+        if (foundNodes.matchedBy(addr) || excludedNodes.match(addr)) {
+          continue;
+        }
+        // The remaining nodes are ones that are referenced by the hosts
+        // files but that we do not know about, ie that we have never
+        // head from. Eg. an entry that is no longer part of the cluster
+        // or a bogus entry was given in the hosts files
+        //
+        // If the host file entry specified the xferPort, we use that.
+        // Otherwise, we guess that it is the default xfer port.
+        // We can't ask the DataNode what it had configured, because it's
+        // dead.
+        DatanodeDescriptor dn = new DatanodeDescriptor(new DatanodeID(addr
+                .getAddress().getHostAddress(), addr.getHostName(), "",
+                addr.getPort() == 0 ? defaultXferPort : addr.getPort(),
+                defaultInfoPort, defaultInfoSecurePort, defaultIpcPort));
+        dn.setLastUpdate(0); // Consider this node dead for reporting
+        nodes.add(dn);
       }
     }
+
     if (LOG.isDebugEnabled()) {
       LOG.debug("getDatanodeListForReport with " +
           "includedNodes = " + hostFileManager.getIncludes() +

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java?rev=1585626&r1=1585625&r2=1585626&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
 Mon Apr  7 23:58:24 2014
@@ -33,6 +33,7 @@ import org.apache.hadoop.test.GenericTes
 import org.apache.hadoop.util.VersionInfo;
 import org.junit.Test;
 
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.Permission;
 
@@ -226,6 +227,7 @@ public class TestDatanodeRegistration {
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
       
doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn("127.0.0.1").when(mockDnReg).getIpAddr();
       doReturn(123).when(mockDnReg).getXferPort();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
@@ -280,6 +282,7 @@ public class TestDatanodeRegistration {
       // Should succeed when software versions are the same and CTimes are the
       // same.
       doReturn(VersionInfo.getVersion()).when(mockDnReg).getSoftwareVersion();
+      doReturn("127.0.0.1").when(mockDnReg).getIpAddr();
       doReturn(123).when(mockDnReg).getXferPort();
       rpcServer.registerDatanode(mockDnReg);
       

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java?rev=1585626&r1=1585625&r2=1585626&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
 Mon Apr  7 23:58:24 2014
@@ -25,9 +25,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.Map;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -45,7 +43,6 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.PathUtils;
@@ -639,149 +636,6 @@ public class TestDecommission {
       assertEquals(bogusIp, info[1].getHostName());
     }
   }
-
-  @Test(timeout=360000)
-  public void testDuplicateHostsEntries() throws IOException,
-      InterruptedException {
-    Configuration hdfsConf = new Configuration(conf);
-    cluster = new MiniDFSCluster.Builder(hdfsConf)
-        .numDataNodes(1).setupHostsFile(true).build();
-    cluster.waitActive();
-    int dnPort = cluster.getDataNodes().get(0).getXferPort();
-
-    // pick some random ports that don't overlap with our DN's port
-    // or with each other.
-    Random random = new Random(System.currentTimeMillis());
-    int port1 = dnPort;
-    while (port1 == dnPort) {
-      port1 = random.nextInt(6000) + 1000;
-    }
-    int port2 = dnPort;
-    while ((port2 == dnPort) || (port2 == port1)) {
-      port2 = random.nextInt(6000) + 1000;
-    }
-
-    // Now empty hosts file and ensure the datanode is disallowed
-    // from talking to namenode, resulting in it's shutdown.
-    ArrayList<String> nodes = new ArrayList<String>();
-
-    // These entries will be de-duped by the NameNode, since they refer
-    // to the same IP address + port combo.
-    nodes.add("127.0.0.1:" + port1);
-    nodes.add("localhost:" + port1);
-    nodes.add("127.0.0.1:" + port1);
-
-    // The following entries should not be de-duped.
-    nodes.add("127.0.0.1:" + port2);
-    nodes.add("127.0.30.1:" + port1);
-    writeConfigFile(hostsFile,  nodes);
-
-    refreshNodes(cluster.getNamesystem(0), hdfsConf);
-
-    DFSClient client = getDfsClient(cluster.getNameNode(0), hdfsConf);
-    DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
-    for (int i = 0 ; i < 5 && info.length != 0; i++) {
-      LOG.info("Waiting for datanode to be marked dead");
-      Thread.sleep(HEARTBEAT_INTERVAL * 1000);
-      info = client.datanodeReport(DatanodeReportType.LIVE);
-    }
-    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 entries in the hosts file.
-    DatanodeInfo deadDns[] = client.datanodeReport(DatanodeReportType.DEAD);
-    HashMap<String, DatanodeInfo> deadByXferAddr =
-        new HashMap<String, DatanodeInfo>();
-    for (DatanodeInfo dn : deadDns) {
-      LOG.info("DEAD DatanodeInfo: xferAddr = " + dn.getXferAddr() +
-          ", ipAddr = " + dn.getIpAddr() +
-          ", hostname = " + dn.getHostName());
-      deadByXferAddr.put(dn.getXferAddr(), dn);
-    }
-    // The real DataNode should be included in the list.
-    String realDnIpPort = cluster.getDataNodes().get(0).
-        getXferAddress().getAddress().getHostAddress() + ":" +
-        cluster.getDataNodes().get(0).getXferPort();
-    Assert.assertNotNull("failed to find real datanode IP " + realDnIpPort,
-        deadByXferAddr.remove(realDnIpPort));
-    // The fake datanode with address 127.0.30.1 should be included in this 
list.
-    Assert.assertNotNull(deadByXferAddr.remove(
-        "127.0.30.1:" + port1));
-    // Now look for the two copies of 127.0.0.1 with port1 and port2.
-    Iterator<Map.Entry<String, DatanodeInfo>> iter =
-            deadByXferAddr.entrySet().iterator();
-    boolean foundPort1 = false, foundPort2 = false;
-    while (iter.hasNext()) {
-      Map.Entry<String, DatanodeInfo> entry = iter.next();
-      DatanodeInfo dn = entry.getValue();
-      if (dn.getXferPort() == port1) {
-        foundPort1 = true;
-        iter.remove();
-      } else if (dn.getXferPort() == port2) {
-        foundPort2 = true;
-        iter.remove();
-      }
-    }
-    Assert.assertTrue("did not find a dead entry with port " + port1,
-        foundPort1);
-    Assert.assertTrue("did not find a dead entry with port " + port2,
-        foundPort2);
-    Assert.assertTrue(deadByXferAddr.isEmpty());
-  }
-
-  @Test(timeout=360000)
-  public void testIncludeByRegistrationName() throws IOException,
-      InterruptedException {
-    Configuration hdfsConf = new Configuration(conf);
-    final String registrationName = "--registration-name--";
-    final String nonExistentDn = "127.0.0.40";
-    hdfsConf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, registrationName);
-    cluster = new MiniDFSCluster.Builder(hdfsConf)
-        .numDataNodes(1).checkDataNodeHostConfig(true)
-        .setupHostsFile(true).build();
-    cluster.waitActive();
-
-    // Set up an includes file that doesn't have our datanode.
-    ArrayList<String> nodes = new ArrayList<String>();
-    nodes.add(nonExistentDn);
-    writeConfigFile(hostsFile,  nodes);
-    refreshNodes(cluster.getNamesystem(0), hdfsConf);
-
-    // Wait for the DN to be marked dead.
-    DFSClient client = getDfsClient(cluster.getNameNode(0), hdfsConf);
-    while (true) {
-      DatanodeInfo info[] = client.datanodeReport(DatanodeReportType.DEAD);
-      if (info.length == 1) {
-        break;
-      }
-      LOG.info("Waiting for datanode to be marked dead");
-      Thread.sleep(HEARTBEAT_INTERVAL * 1000);
-    }
-
-    // Use a non-empty include file with our registration name.
-    // It should work.
-    int dnPort = cluster.getDataNodes().get(0).getXferPort();
-    nodes = new ArrayList<String>();
-    nodes.add(registrationName + ":" + dnPort);
-    writeConfigFile(hostsFile,  nodes);
-    refreshNodes(cluster.getNamesystem(0), hdfsConf);
-    cluster.restartDataNode(0);
-
-    // Wait for the DN to come back.
-    while (true) {
-      DatanodeInfo info[] = client.datanodeReport(DatanodeReportType.LIVE);
-      if (info.length == 1) {
-        Assert.assertFalse(info[0].isDecommissioned());
-        Assert.assertFalse(info[0].isDecommissionInProgress());
-        assertEquals(registrationName, info[0].getHostName());
-        break;
-      }
-      LOG.info("Waiting for datanode to come back");
-      Thread.sleep(HEARTBEAT_INTERVAL * 1000);
-    }
-  }
   
   @Test(timeout=120000)
   public void testDecommissionWithOpenfile() throws IOException, 
InterruptedException {

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=1585626&r1=1585625&r2=1585626&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
 Mon Apr  7 23:58:24 2014
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -895,16 +896,9 @@ public class NNThroughputBenchmark imple
     long[] blockReportList;
     final int dnIdx;
 
-    /**
-     * Return a a 6 digit integer port.
-     * This is necessary in order to provide lexocographic ordering.
-     * Host names are all the same, the ordering goes by port numbers.
-     */
     private static int getNodePort(int num) throws IOException {
-      int port = 100000 + num;
-      if (String.valueOf(port).length() > 6) {
-        throw new IOException("Too many data-nodes");
-      }
+      int port = 1 + num;
+      Preconditions.checkState(port < Short.MAX_VALUE);
       return port;
     }
 


Reply via email to