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 at...@apache.org on 2011/06/25 02:39:29 UTC

svn commit: r1139473 - in /hadoop/common/trunk/hdfs: CHANGES.txt src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java

Author: atm
Date: Sat Jun 25 00:39:29 2011
New Revision: 1139473

URL: http://svn.apache.org/viewvc?rev=1139473&view=rev
Log:
HDFS-1321. If service port and main port are the same, there is no clear log message explaining the issue. (Jim Plush via atm)

Added:
    hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
Modified:
    hadoop/common/trunk/hdfs/CHANGES.txt
    hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

Modified: hadoop/common/trunk/hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/CHANGES.txt?rev=1139473&r1=1139472&r2=1139473&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hdfs/CHANGES.txt Sat Jun 25 00:39:29 2011
@@ -777,6 +777,9 @@ Trunk (unreleased changes)
 
     HDFS-2082. SecondaryNameNode web interface doesn't show the right info. (atm)
 
+    HDFS-1321. If service port and main port are the same, there is no clear
+    log message explaining the issue. (Jim Plush via atm)
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1139473&r1=1139472&r2=1139473&view=diff
==============================================================================
--- hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/trunk/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Sat Jun 25 00:39:29 2011
@@ -437,6 +437,13 @@ public class NameNode implements Namenod
     // The rpc-server port can be ephemeral... ensure we have the correct info
     this.rpcAddress = this.server.getListenerAddress(); 
     setRpcServerAddress(conf);
+    
+    try {
+      validateConfigurationSettings(conf);
+    } catch (IOException e) {
+      LOG.fatal(e.toString());
+      throw e;
+    }
 
     activate(conf);
     LOG.info(getRole() + " up at: " + rpcAddress);
@@ -444,6 +451,28 @@ public class NameNode implements Namenod
       LOG.info(getRole() + " service server is up at: " + serviceRPCAddress); 
     }
   }
+  
+  /**
+   * Verifies that the final Configuration Settings look ok for the NameNode to
+   * properly start up
+   * Things to check for include:
+   * - HTTP Server Port does not equal the RPC Server Port
+   * @param conf
+   * @throws IOException
+   */
+  protected void validateConfigurationSettings(final Configuration conf) 
+      throws IOException {
+    // check to make sure the web port and rpc port do not match 
+    if(getHttpServerAddress(conf).getPort() 
+        == getRpcServerAddress(conf).getPort()) {
+      String errMsg = "dfs.namenode.rpc-address " +
+          "("+ getRpcServerAddress(conf) + ") and " +
+          "dfs.namenode.http-address ("+ getHttpServerAddress(conf) + ") " +
+          "configuration keys are bound to the same port, unable to start " +
+          "NameNode. Port: " + getRpcServerAddress(conf).getPort();
+      throw new IOException(errMsg);
+    } 
+  }
 
   /**
    * Activate name-node servers and threads.

Added: hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java?rev=1139473&view=auto
==============================================================================
--- hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java (added)
+++ hadoop/common/trunk/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java Sat Jun 25 00:39:29 2011
@@ -0,0 +1,74 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+import java.io.IOException;
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+
+/**
+ * This class tests the validation of the configuration object when passed 
+ * to the NameNode
+ */
+public class TestValidateConfigurationSettings {
+
+  /**
+   * Tests setting the rpc port to the same as the web port to test that 
+   * an exception
+   * is thrown when trying to re-use the same port
+   */
+  @Test
+  public void testThatMatchingRPCandHttpPortsThrowException() 
+      throws IOException {
+
+    Configuration conf = new HdfsConfiguration();
+    // set both of these to port 9000, should fail
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:9000"); 
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
+    DFSTestUtil.formatNameNode(conf);
+    try {
+      NameNode nameNode = new NameNode(conf);
+      Assert.fail("Should have throw the exception since the ports match");
+    } catch (IOException e) {
+      // verify we're getting the right IOException
+      assertTrue(e.toString().contains("dfs.namenode.rpc-address (")); 
+      System.out.println("Got expected exception: " + e.toString());
+    }
+  }
+
+  /**
+   * Tests setting the rpc port to a different as the web port that an 
+   * exception is NOT thrown 
+   */
+  @Test
+  public void testThatDifferentRPCandHttpPortsAreOK() 
+      throws IOException {
+
+    Configuration conf = new HdfsConfiguration();
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:8000");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
+    DFSTestUtil.formatNameNode(conf);
+    NameNode nameNode = new NameNode(conf); // should be OK!
+  }
+}