浏览代码

HDFS-3139. svn merge -c 1306549 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1306553 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 年之前
父节点
当前提交
ee241b9987

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

@@ -785,6 +785,8 @@ Release 0.23.1 - 2012-02-17
     HDFS-208. name node should warn if only one dir is listed in dfs.name.dir.
     (Uma Maheswara Rao G via eli)
 
+    HDFS-3139. Minor Datanode logging improvement. (eli)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -28,20 +28,20 @@ import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableComparable;
 
 /**
- * DatanodeID is composed of the data node 
- * name (hostname:portNumber) and the data storage ID, 
- * which it currently represents.
- * 
+ * This class represents the primary identifier for a Datanode.
+ * Datanodes are identified by how they can be contacted (hostname
+ * and ports) and their storage ID, a unique number that associates
+ * the Datanodes blocks with a particular Datanode.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class DatanodeID implements WritableComparable<DatanodeID> {
   public static final DatanodeID[] EMPTY_ARRAY = {}; 
 
-  public String name;      /// hostname:portNumber
-  public String storageID; /// unique per cluster storageID
-  protected int infoPort;     /// the port where the infoserver is running
-  public int ipcPort;     /// the port where the ipc server is running
+  public String name;       // hostname:port (data transfer port)
+  public String storageID;  // unique per cluster storageID
+  protected int infoPort;   // info server port
+  public int ipcPort;       // ipc server port
 
   /** Equivalent to DatanodeID(""). */
   public DatanodeID() {this("");}

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -37,9 +37,9 @@ import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.StringUtils;
 
 /** 
- * DatanodeInfo represents the status of a DataNode.
- * This object is used for communication in the
- * Datanode Protocol and the Client Protocol.
+ * This class extends the primary identifier of a Datanode with ephemeral
+ * state, eg usage information, current administrative state, and the
+ * network location that is communicated to clients.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -52,12 +52,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
   protected int xceiverCount;
   protected String location = NetworkTopology.DEFAULT_RACK;
 
-  /** HostName as supplied by the datanode during registration as its 
-   * name. Namenode uses datanode IP address as the name.
-   */
+  // The FQDN of the IP associated with the Datanode's hostname
   protected String hostName = null;
   
-  // administrative states of a datanode
+  // Datanode administrative states
   public enum AdminStates {
     NORMAL("In Service"), 
     DECOMMISSION_INPROGRESS("Decommission In Progress"), 
@@ -241,12 +239,14 @@ public class DatanodeInfo extends DatanodeID implements Node {
     long nonDFSUsed = getNonDfsUsed();
     float usedPercent = getDfsUsedPercent();
     float remainingPercent = getRemainingPercent();
-    String hostName = NetUtils.getHostNameOfIP(name);
+    String lookupName = NetUtils.getHostNameOfIP(name);
 
     buffer.append("Name: "+ name);
-    if(hostName != null)
-      buffer.append(" (" + hostName + ")");
+    if (lookupName != null) {
+      buffer.append(" (" + lookupName + ")");
+    }
     buffer.append("\n");
+    buffer.append("Hostname: " + getHostName() + "\n");
 
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append("Rack: "+location+"\n");

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

@@ -38,16 +38,13 @@ import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 
-/**************************************************
- * DatanodeDescriptor tracks stats on a given DataNode, such as
- * available storage capacity, last update time, etc., and maintains a
- * set of blocks stored on the datanode.
- *
- * This data structure is internal to the namenode. It is *not* sent
- * over-the-wire to the Client or the Datanodes. Neither is it stored
- * persistently in the fsImage.
- **************************************************/
+/**
+ * This class extends the DatanodeInfo class with ephemeral information (eg
+ * health, capacity, what blocks are associated with the Datanode) that is
+ * private to the Namenode, ie this class is not exposed to clients.
+ */
 @InterfaceAudience.Private
+@InterfaceStability.Evolving
 public class DatanodeDescriptor extends DatanodeInfo {
   
   // Stores status of decommissioning.
@@ -590,14 +587,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * @return Blanacer bandwidth in bytes per second for this datanode.
+   * @return balancer bandwidth in bytes per second for this datanode
    */
   public long getBalancerBandwidth() {
     return this.bandwidth;
   }
 
   /**
-   * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
+   * @param bandwidth balancer bandwidth in bytes per second for this datanode
    */
   public void setBalancerBandwidth(long bandwidth) {
     this.bandwidth = bandwidth;

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

@@ -335,9 +335,7 @@ public class DataNode extends Configured
        : new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
            conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
            secureResources.getListener());
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
-    }
+    LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
                                                DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
@@ -403,7 +401,8 @@ public class DataNode extends Configured
         .newReflectiveBlockingService(interDatanodeProtocolXlator);
     DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
         ipcServer);
-    
+    LOG.info("Opened IPC server at " + ipcServer.getListenerAddress());
+
     // set service-level authorization security policy
     if (conf.getBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
@@ -491,14 +490,14 @@ public class DataNode extends Configured
   }
   
   private void initDataXceiver(Configuration conf) throws IOException {
-    InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
+    InetSocketAddress streamingAddr = DataNode.getStreamingAddr(conf);
 
     // find free port or use privileged port provided
     ServerSocket ss;
     if(secureResources == null) {
       ss = (dnConf.socketWriteTimeout > 0) ? 
           ServerSocketChannel.open().socket() : new ServerSocket();
-          Server.bind(ss, socAddr, 0);
+          Server.bind(ss, streamingAddr, 0);
     } else {
       ss = secureResources.getStreamingSocket();
     }
@@ -507,8 +506,7 @@ public class DataNode extends Configured
     int tmpPort = ss.getLocalPort();
     selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
                                      tmpPort);
-    LOG.info("Opened info server at " + tmpPort);
-      
+    LOG.info("Opened streaming server at " + selfAddr);
     this.threadGroup = new ThreadGroup("dataXceiverServer");
     this.dataXceiverServer = new Daemon(threadGroup, 
         new DataXceiverServer(ss, conf, this));

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -69,18 +69,19 @@ public class SecureDataNodeStarter implements Daemon {
     args = context.getArguments();
     
     // Obtain secure port for data streaming to datanode
-    InetSocketAddress socAddr = DataNode.getStreamingAddr(conf);
+    InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
     int socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
         HdfsServerConstants.WRITE_TIMEOUT);
     
     ServerSocket ss = (socketWriteTimeout > 0) ? 
         ServerSocketChannel.open().socket() : new ServerSocket();
-    ss.bind(socAddr, 0);
+    ss.bind(streamingAddr, 0);
     
     // Check that we got the port we need
-    if(ss.getLocalPort() != socAddr.getPort())
+    if (ss.getLocalPort() != streamingAddr.getPort()) {
       throw new RuntimeException("Unable to bind on specified streaming port in secure " +
-      		"context. Needed " + socAddr.getPort() + ", got " + ss.getLocalPort());
+      		"context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+    }
 
     // Obtain secure listener for web server
     SelectChannelConnector listener = 
@@ -90,15 +91,18 @@ public class SecureDataNodeStarter implements Daemon {
     listener.setPort(infoSocAddr.getPort());
     // Open listener here in order to bind to port as root
     listener.open(); 
-    if(listener.getPort() != infoSocAddr.getPort())
+    if (listener.getPort() != infoSocAddr.getPort()) {
       throw new RuntimeException("Unable to bind on specified info port in secure " +
-          "context. Needed " + socAddr.getPort() + ", got " + ss.getLocalPort());
+          "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+    }
     System.err.println("Successfully obtained privileged resources (streaming port = "
         + ss + " ) (http listener port = " + listener.getConnection() +")");
     
-    if(ss.getLocalPort() >= 1023 || listener.getPort() >= 1023)
+    if (ss.getLocalPort() >= 1023 || listener.getPort() >= 1023) {
       throw new RuntimeException("Cannot start secure datanode with unprivileged ports");
-    
+    }
+    System.err.println("Opened streaming server at " + streamingAddr);
+    System.err.println("Opened info server at " + infoSocAddr);
     resources = new SecureResources(ss, listener);
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java

@@ -52,7 +52,7 @@ public class TestDFSAddressConfig extends TestCase {
 
     String selfSocketAddr = dn.getSelfAddr().toString();
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
-    assertTrue(selfSocketAddr.startsWith("/127.0.0.1:"));
+    assertTrue(selfSocketAddr.contains("/127.0.0.1:"));
 
     /*-------------------------------------------------------------------------
      * Shut down the datanodes, reconfigure, and bring them back up.
@@ -78,7 +78,7 @@ public class TestDFSAddressConfig extends TestCase {
     selfSocketAddr = dn.getSelfAddr().toString();
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
     // assert that default self socket address is 127.0.0.1
-    assertTrue(selfSocketAddr.startsWith("/127.0.0.1:"));
+    assertTrue(selfSocketAddr.contains("/127.0.0.1:"));
 
     /*-------------------------------------------------------------------------
      * Shut down the datanodes, reconfigure, and bring them back up.
@@ -103,7 +103,7 @@ public class TestDFSAddressConfig extends TestCase {
     selfSocketAddr = dn.getSelfAddr().toString();
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
     // assert that default self socket address is 0.0.0.0
-    assertTrue(selfSocketAddr.startsWith("/0.0.0.0:"));
+    assertTrue(selfSocketAddr.contains("/0.0.0.0:"));
 
     cluster.shutdown();
   }