You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ki...@apache.org on 2013/08/27 22:53:32 UTC

svn commit: r1517981 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/resources/ src/test/java/org/apache/hadoop/hdfs/server/namenode/

Author: kihwal
Date: Tue Aug 27 20:53:31 2013
New Revision: 1517981

URL: http://svn.apache.org/r1517981
Log:
HDFS-5128. Allow multiple net interfaces to be used with HA namenode RPC server. Contributed by Kihwal Lee.

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java   (with props)
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1517981&r1=1517980&r2=1517981&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Tue Aug 27 20:53:31 2013
@@ -336,6 +336,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-3245. Add metrics and web UI for cluster version summary. (Ravi
     Prakash via kihwal)
 
+    HDFS-5128. Allow multiple net interfaces to be used with HA namenode RPC
+    server. (kihwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1517981&r1=1517980&r2=1517981&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Tue Aug 27 20:53:31 2013
@@ -104,7 +104,9 @@ public class DFSConfigKeys extends Commo
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTP_PORT_DEFAULT;
   public static final String  DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
+  public static final String  DFS_NAMENODE_RPC_BIND_HOST_KEY = "dfs.namenode.rpc-bind-host";
   public static final String  DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.servicerpc-address";
+  public static final String  DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY = "dfs.namenode.servicerpc-bind-host";
   public static final String  DFS_NAMENODE_MAX_OBJECTS_KEY = "dfs.namenode.max.objects";
   public static final long    DFS_NAMENODE_MAX_OBJECTS_DEFAULT = 0;
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_KEY = "dfs.namenode.safemode.extension";

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1517981&r1=1517980&r2=1517981&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Tue Aug 27 20:53:31 2013
@@ -166,12 +166,14 @@ public class NameNode implements NameNod
    */
   public static final String[] NAMENODE_SPECIFIC_KEYS = {
     DFS_NAMENODE_RPC_ADDRESS_KEY,
+    DFS_NAMENODE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_NAME_DIR_KEY,
     DFS_NAMENODE_EDITS_DIR_KEY,
     DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+    DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
@@ -387,6 +389,28 @@ public class NameNode implements NameNod
     return getAddress(conf);
   }
   
+  /** Given a configuration get the bind host of the service rpc server
+   *  If the bind host is not configured returns null.
+   */
+  protected String getServiceRpcServerBindHost(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY);
+    if (addr == null || addr.isEmpty()) {
+      return null;
+    }
+    return addr;
+  }
+
+  /** Given a configuration get the bind host of the client rpc server
+   *  If the bind host is not configured returns null.
+   */
+  protected String getRpcServerBindHost(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_NAMENODE_RPC_BIND_HOST_KEY);
+    if (addr == null || addr.isEmpty()) {
+      return null;
+    }
+    return addr;
+  }
+   
   /**
    * Modifies the configuration passed to contain the service rpc address setting
    */

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java?rev=1517981&r1=1517980&r2=1517981&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Tue Aug 27 20:53:31 2013
@@ -136,6 +136,7 @@ import org.apache.hadoop.tools.protocolP
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 
 /**
@@ -218,6 +219,13 @@ class NameNodeRpcServer implements Namen
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     if (serviceRpcAddr != null) {
+      String bindHost = nn.getServiceRpcServerBindHost(conf);
+      if (bindHost == null) {
+        bindHost = serviceRpcAddr.getHostName();
+      }
+      LOG.info("Service RPC server is binding to " + bindHost + ":" +
+          serviceRpcAddr.getPort());
+
       int serviceHandlerCount =
         conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
                     DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
@@ -225,7 +233,7 @@ class NameNodeRpcServer implements Namen
           .setProtocol(
               org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
           .setInstance(clientNNPbService)
-          .setBindAddress(serviceRpcAddr.getHostName())
+          .setBindAddress(bindHost)
           .setPort(serviceRpcAddr.getPort())
           .setNumHandlers(serviceHandlerCount)
           .setVerbose(false)
@@ -246,7 +254,10 @@ class NameNodeRpcServer implements Namen
       DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
           getUserMappingService, serviceRpcServer);
   
-      serviceRPCAddress = serviceRpcServer.getListenerAddress();
+      // Update the address with the correct port
+      InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
+      serviceRPCAddress = new InetSocketAddress(
+            serviceRpcAddr.getHostName(), listenAddr.getPort());
       nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
     } else {
       serviceRpcServer = null;
@@ -254,11 +265,17 @@ class NameNodeRpcServer implements Namen
     }
 
     InetSocketAddress rpcAddr = nn.getRpcServerAddress(conf);
+    String bindHost = nn.getRpcServerBindHost(conf);
+    if (bindHost == null) {
+      bindHost = rpcAddr.getHostName();
+    }
+    LOG.info("RPC server is binding to " + bindHost + ":" + rpcAddr.getPort());
+
     clientRpcServer = new RPC.Builder(conf)
         .setProtocol(
             org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
         .setInstance(clientNNPbService)
-        .setBindAddress(rpcAddr.getHostName())
+        .setBindAddress(bindHost)
         .setPort(rpcAddr.getPort())
         .setNumHandlers(handlerCount)
         .setVerbose(false)
@@ -290,7 +307,9 @@ class NameNodeRpcServer implements Namen
     }
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    clientRpcAddress = clientRpcServer.getListenerAddress();
+    InetSocketAddress listenAddr = clientRpcServer.getListenerAddress();
+      clientRpcAddress = new InetSocketAddress(
+          rpcAddr.getHostName(), listenAddr.getPort());
     nn.setRpcServerAddress(conf, clientRpcAddress);
     
     minimumDataNodeVersion = conf.get(
@@ -314,6 +333,12 @@ class NameNodeRpcServer implements Namen
         NSQuotaExceededException.class,
         DSQuotaExceededException.class);
  }
+
+  /** Allow access to the client RPC server for testing */
+  @VisibleForTesting
+  RPC.Server getClientRpcServer() {
+    return clientRpcServer;
+  }
   
   /**
    * Start client and service RPC servers.

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml?rev=1517981&r1=1517980&r2=1517981&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml Tue Aug 27 20:53:31 2013
@@ -52,6 +52,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.rpc-bind-host</name>
+  <value></value>
+  <description>
+    The actual address the server will bind to. If this optional address is
+    set, the RPC server will bind to this address and the port specified in
+    dfs.namenode.rpc-address for the RPC server. It can also be specified
+    per name node or name service for HA/Federation. This is most useful for
+    making name node listen to all interfaces by setting to 0.0.0.0.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.servicerpc-address</name>
   <value></value>
   <description>
@@ -65,6 +77,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.servicerpc-bind-host</name>
+  <value></value>
+  <description>
+    The actual address the server will bind to. If this optional address is
+    set, the service RPC server will bind to this address and the port 
+    specified in dfs.namenode.servicerpc-address. It can also be specified
+    per name node or name service for HA/Federation. This is most useful for
+    making name node listen to all interfaces by setting to 0.0.0.0.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.secondary.http-address</name>
   <value>0.0.0.0:50090</value>
   <description>

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java?rev=1517981&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java Tue Aug 27 20:53:31 2013
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*
+ * Test the MiniDFSCluster functionality that allows "dfs.datanode.address",
+ * "dfs.datanode.http.address", and "dfs.datanode.ipc.address" to be
+ * configurable. The MiniDFSCluster.startDataNodes() API now has a parameter
+ * that will check these properties if told to do so.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.Test;
+
+public class TestNameNodeRpcServer {
+
+  @Test
+  public void testNamenodeRpcBindAny() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+
+    // The name node in MiniDFSCluster only binds to 127.0.0.1.
+    // We can set the bind address to 0.0.0.0 to make it listen
+    // to all interfaces.
+    conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY, "0.0.0.0");
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      assertEquals("0.0.0.0", ((NameNodeRpcServer)cluster.getNameNodeRpc())
+          .getClientRpcServer().getListenerAddress().getHostName());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      // Reset the config
+      conf.unset(DFS_NAMENODE_RPC_BIND_HOST_KEY);
+    }
+  }
+}
+

Propchange: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java
------------------------------------------------------------------------------
    svn:eol-style = native