Sfoglia il codice sorgente

HADOOP-2404. Fix backwards compatability with hadoop-0.15 configuration files
that was broken by HADOOP-2185.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@616899 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 17 anni fa
parent
commit
3a17875767

+ 29 - 2
CHANGES.txt

@@ -36,8 +36,32 @@ Release 0.16.0 - 2008-02-04
     HADOOP-2184.  RPC Support for user permissions and authentication.
     (Raghu Angadi via dhruba)
 
-    HADOOP-2185.  RPC Server uses any available port if the specified port is
-    zero. Otherwise it uses the specified port.
+    HADOOP-2185.  RPC Server uses any available port if the specified
+    port is zero. Otherwise it uses the specified port. Also combines
+    the configuration attributes for the servers' bind address and
+    port from "x.x.x.x" and "y" to "x.x.x.x:y". 
+    Deprecated configuration variables:
+      dfs.info.bindAddress
+      dfs.info.port
+      dfs.datanode.bindAddress
+      dfs.datanode.port
+      dfs.datanode.info.bindAdress
+      dfs.datanode.info.port
+      dfs.secondary.info.bindAddress
+      dfs.secondary.info.port
+      mapred.job.tracker.info.bindAddress
+      mapred.job.tracker.info.port
+      mapred.task.tracker.report.bindAddress
+      tasktracker.http.bindAddress
+      tasktracker.http.port
+    New configuration variables (post HADOOP-2404):
+      dfs.secondary.http.address
+      dfs.datanode.address
+      dfs.datanode.http.address
+      dfs.http.address
+      mapred.job.tracker.http.address
+      mapred.task.tracker.report.address
+      mapred.task.tracker.http.address
     (Konstantin Shvachko via dhruba)
 
     HADOOP-2401.  Only the current leaseholder can abandon a block for
@@ -623,6 +647,9 @@ Release 0.16.0 - 2008-02-04
 
     HADOOP-2732. Fix bug in path globbing.  (Hairong Kuang via nigel)
 
+    HADOOP-2404. Fix backwards compatability with hadoop-0.15 configuration
+    files that was broken by HADOOP-2185. (omalley)
+    
 Release 0.15.3 - 2008-01-18
 
   BUG FIXES

+ 16 - 14
conf/hadoop-default.xml

@@ -184,34 +184,34 @@ creations/deletions), or "all".</description>
 </property>
 
 <property>
-  <name>dfs.secondary.http.bindAddress</name>
+  <name>dfs.secondary.http.address</name>
   <value>0.0.0.0:50090</value>
   <description>
-    The secondary namenode http server bind address and port.
+    The secondary namenode http server address and port.
     If the port is 0 then the server will start on a free port.
   </description>
 </property>
 
 <property>
-  <name>dfs.datanode.bindAddress</name>
+  <name>dfs.datanode.address</name>
   <value>0.0.0.0:50010</value>
   <description>
-    The address where the datanode will listen to.
+    The address where the datanode server will listen to.
     If the port is 0 then the server will start on a free port.
   </description>
 </property>
 
 <property>
-  <name>dfs.datanode.http.bindAddress</name>
+  <name>dfs.datanode.http.address</name>
   <value>0.0.0.0:50075</value>
   <description>
-    The datanode http server bind address and port.
+    The datanode http server address and port.
     If the port is 0 then the server will start on a free port.
   </description>
 </property>
 
 <property>
-  <name>dfs.http.bindAddress</name>
+  <name>dfs.http.address</name>
   <value>0.0.0.0:50070</value>
   <description>
     The address and the base port where the dfs namenode web ui will listen on.
@@ -498,10 +498,10 @@ creations/deletions), or "all".</description>
 </property>
 
 <property>
-  <name>mapred.job.tracker.http.bindAddress</name>
+  <name>mapred.job.tracker.http.address</name>
   <value>0.0.0.0:50030</value>
   <description>
-    The job tracker http server bind address and port.
+    The job tracker http server address and port the server will listen on.
     If the port is 0 then the server will start on a free port.
   </description>
 </property>
@@ -516,10 +516,12 @@ creations/deletions), or "all".</description>
 </property>
 
 <property>
-  <name>mapred.task.tracker.report.bindAddress</name>
+  <name>mapred.task.tracker.report.address</name>
   <value>127.0.0.1:0</value>
-  <description>The interface that task processes use to communicate
-  with their parent tasktracker process.</description>
+  <description>The interface and port that task tracker server listens on. 
+  Since it is only connected to by the tasks, it uses the local interface.
+  EXPERT ONLY. Should only be changed if your host does not have the loopback 
+  interface.</description>
 </property>
 
 <property>
@@ -747,10 +749,10 @@ creations/deletions), or "all".</description>
 </property>
 
 <property>
-  <name>mapred.task.tracker.http.bindAddress</name>
+  <name>mapred.task.tracker.http.address</name>
   <value>0.0.0.0:50060</value>
   <description>
-    The task tracker http server bind address and port.
+    The task tracker http server address and port.
     If the port is 0 then the server will start on a free port.
   </description>
 </property>

+ 3 - 1
src/java/org/apache/hadoop/dfs/DFSck.java

@@ -26,6 +26,7 @@ import java.net.URLEncoder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -66,7 +67,8 @@ public class DFSck extends Configured implements Tool {
   }
   
   private String getInfoServer() throws IOException {
-    return getConf().get("dfs.http.bindAddress", "0.0.0.0:50070");
+    return NetUtils.getServerAddress(getConf(), "dfs.info.bindAddress", 
+                                     "dfs.info.port", "dfs.http.address");
   }
   
   /**

+ 10 - 2
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -192,7 +192,11 @@ public class DataNode implements FSConstants, Runnable {
     this.estimateBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     this.socketTimeout =  conf.getInt("dfs.socket.timeout",
                                       FSConstants.READ_TIMEOUT);
-    String address = conf.get("dfs.datanode.bindAddress", "0.0.0.0:50010");
+    String address = 
+      NetUtils.getServerAddress(conf,
+                                "dfs.datanode.bindAddress", 
+                                "dfs.datanode.port",
+                                "dfs.datanode.address");
     InetSocketAddress socAddr = NetUtils.createSocketAddr(address);
     int tmpPort = socAddr.getPort();
     storage = new DataStorage();
@@ -284,7 +288,11 @@ public class DataNode implements FSConstants, Runnable {
     }
     
     //create a servlet to serve full-file content
-    String infoAddr = conf.get("dfs.datanode.http.bindAddress", "0.0.0.0:50075");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, 
+                              "dfs.datanode.info.bindAddress", 
+                              "dfs.datanode.info.port",
+                              "dfs.datanode.http.address");
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();

+ 5 - 3
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -276,7 +276,9 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     this.dnthread = new Daemon(new DecommissionedMonitor());
     dnthread.start();
 
-    String infoAddr = conf.get("dfs.http.bindAddress", "0.0.0.0:50070");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
+                                "dfs.info.port", "dfs.http.address");
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
@@ -293,8 +295,8 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
 
     // The web-server port can be ephemeral... ensure we have the correct info
     this.infoPort = this.infoServer.getPort();
-    conf.set("dfs.http.bindAddress", infoHost + ":" + infoPort); 
-    LOG.info("Web-server up at: " + conf.get("dfs.http.bindAddress"));
+    conf.set("dfs.http.address", infoHost + ":" + infoPort);
+    LOG.info("Web-server up at: " + infoHost + ":" + infoPort);
   }
 
   static Collection<File> getNamespaceDirs(Configuration conf) {

+ 5 - 1
src/java/org/apache/hadoop/dfs/HftpFileSystem.java

@@ -82,7 +82,11 @@ public class HftpFileSystem extends FileSystem {
     this.fsport = name.getPort();
     if(fsport >= 0)
       return;
-    String infoAddr = conf.get("dfs.http.bindAddress", "0.0.0.0:50070");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, 
+                                "dfs.info.bindAddress", 
+                                "dfs.info.port", 
+                                "dfs.http.address");
     this.fsport = NetUtils.createSocketAddr(infoAddr).getPort();
   }
 

+ 8 - 5
src/java/org/apache/hadoop/dfs/SecondaryNameNode.java

@@ -101,8 +101,11 @@ public class SecondaryNameNode implements FSConstants, Runnable {
     //
     // initialize the webserver for uploading files.
     //
-    String infoAddr = conf.get("dfs.secondary.http.bindAddress", 
-                                "0.0.0.0:50090");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, 
+                                "dfs.secondary.info.bindAddress",
+                                "dfs.secondary.info.port",
+                                "dfs.secondary.http.address");
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     infoBindAddress = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
@@ -116,8 +119,7 @@ public class SecondaryNameNode implements FSConstants, Runnable {
     // The web-server port can be ephemeral... ensure we have the correct info
     infoPort = infoServer.getPort();
     conf.set("dfs.secondary.http.bindAddress", infoBindAddress + ":" +infoPort); 
-    LOG.info("Secondary Web-server up at: " 
-              + conf.get("dfs.secondary.http.bindAddress"));
+    LOG.info("Secondary Web-server up at: " + infoBindAddress + ":" +infoPort);
 
     //
     // Initialize other scheduling parameters from the configuration
@@ -253,7 +255,8 @@ public class SecondaryNameNode implements FSConstants, Runnable {
     if (fsName.equals("local")) {
       throw new IOException("This is not a DFS");
     }
-    return conf.get("dfs.http.bindAddress", "0.0.0.0:50070");
+    return NetUtils.getServerAddress(conf, "dfs.info.bindAddress", 
+                                     "dfs.info.port", "dfs.http.address");
   }
 
   /*

+ 4 - 2
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -647,8 +647,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       }
     }
 
-    String infoAddr = conf.get("mapred.job.tracker.http.bindAddress",
-                                "0.0.0.0:50030");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf, "mapred.job.tracker.info.bindAddress",
+                                "mapred.job.tracker.info.port",
+                                "mapred.job.tracker.http.address");
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String infoBindAddress = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();

+ 9 - 3
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -416,7 +416,10 @@ public class TaskTracker
     
     // bind address
     String address = 
-      this.fConf.get("mapred.task.tracker.report.bindAddress", "127.0.0.1:0");
+      NetUtils.getServerAddress(fConf,
+                                "mapred.task.tracker.report.bindAddress", 
+                                "mapred.task.tracker.report.port", 
+                                "mapred.task.tracker.report.address");
     InetSocketAddress socAddr = NetUtils.createSocketAddr(address);
     String bindAddress = socAddr.getHostName();
     int tmpPort = socAddr.getPort();
@@ -803,8 +806,11 @@ public class TaskTracker
     this.jobTrackAddr = JobTracker.getAddress(conf);
     this.mapOutputFile = new MapOutputFile();
     this.mapOutputFile.setConf(conf);
-    String infoAddr = conf.get("mapred.task.tracker.http.bindAddress", 
-                                "0.0.0.0:50060");
+    String infoAddr = 
+      NetUtils.getServerAddress(conf,
+                                "tasktracker.http.bindAddress", 
+                                "tasktracker.http.port",
+                                "mapred.task.tracker.http.address");
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String httpBindAddress = infoSocAddr.getHostName();
     int httpPort = infoSocAddr.getPort();

+ 43 - 0
src/java/org/apache/hadoop/net/NetUtils.java

@@ -21,12 +21,15 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import javax.net.SocketFactory;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.util.ReflectionUtils;
 
 public class NetUtils {
+  private static final Log LOG = LogFactory.getLog(NetUtils.class);
 
   /**
    * Get the socket factory for the given class according to its
@@ -121,4 +124,44 @@ public class NetUtils {
   
     return new InetSocketAddress(hostname, port);
   }
+
+  /**
+   * Handle the transition from pairs of attributes specifying a host and port
+   * to a single colon separated one.
+   * @param conf the configuration to check
+   * @param oldBindAddressName the old address attribute name
+   * @param oldPortName the old port attribute name
+   * @param newBindAddressName the new combined name
+   * @return the complete address from the configuration
+   */
+  @Deprecated
+  public static String getServerAddress(Configuration conf,
+                                        String oldBindAddressName,
+                                        String oldPortName,
+                                        String newBindAddressName) {
+    String oldAddr = conf.get(oldBindAddressName);
+    String oldPort = conf.get(oldPortName);
+    String newAddrPort = conf.get(newBindAddressName);
+    if (oldAddr == null && oldPort == null) {
+      return newAddrPort;
+    }
+    String[] newAddrPortParts = newAddrPort.split(":",2);
+    if (newAddrPortParts.length != 2) {
+      throw new IllegalArgumentException("Invalid address/port: " + 
+                                         newAddrPort);
+    }
+    if (oldAddr == null) {
+      oldAddr = newAddrPortParts[0];
+    } else {
+      LOG.warn("Configuration parameter " + oldBindAddressName +
+               " is deprecated. Use " + newBindAddressName + " instead.");
+    }
+    if (oldPort == null) {
+      oldPort = newAddrPortParts[1];
+    } else {
+      LOG.warn("Configuration parameter " + oldPortName +
+               " is deprecated. Use " + newBindAddressName + " instead.");      
+    }
+    return oldAddr + ":" + oldPort;
+  }
 }

+ 3 - 3
src/test/org/apache/hadoop/dfs/MiniDFSCluster.java

@@ -168,7 +168,7 @@ public class MiniDFSCluster {
     
     // Setup the NameNode configuration
     conf.set("fs.default.name", "localhost:"+ Integer.toString(nameNodePort));
-    conf.set("dfs.http.bindAddress", "0.0.0.0:0");  
+    conf.set("dfs.http.address", "0.0.0.0:0");  
     if (manageDfsDirs) {
       conf.set("dfs.name.dir", new File(base_dir, "name1").getPath()+","+
                new File(base_dir, "name2").getPath());
@@ -262,8 +262,8 @@ public class MiniDFSCluster {
     }
 
     // Set up the right ports for the datanodes
-    conf.set("dfs.datanode.bindAddress", "0.0.0.0:0");
-    conf.set("dfs.datanode.http.bindAddress", "0.0.0.0:0");
+    conf.set("dfs.datanode.address", "0.0.0.0:0");
+    conf.set("dfs.datanode.http.address", "0.0.0.0:0");
     
     String[] args = (operation == null ||
                      operation == StartupOption.FORMAT ||

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestCheckpoint.java

@@ -331,7 +331,7 @@ public class TestCheckpoint extends TestCase {
     Collection<File> namedirs = null;
 
     Configuration conf = new Configuration();
-    conf.set("dfs.secondary.http.bindAddress", "0.0.0.0:0");
+    conf.set("dfs.secondary.http.address", "0.0.0.0:0");
     replication = (short)conf.getInt("dfs.replication", 3);  
     MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
     cluster.waitActive();

+ 16 - 12
src/test/org/apache/hadoop/dfs/TestHDFSServerPorts.java

@@ -54,7 +54,7 @@ public class TestHDFSServerPorts extends TestCase {
     config = new Configuration();
     config.set("dfs.name.dir", new File(hdfsDir, "name1").getPath());
     config.set("fs.default.name", NAME_NODE_HOST + "0");
-    config.set("dfs.http.bindAddress", NAME_NODE_HTTP_HOST + "0");
+    config.set("dfs.http.address", NAME_NODE_HTTP_HOST + "0");
     NameNode.format(config);
 
     String[] args = new String[] {};
@@ -144,7 +144,7 @@ public class TestHDFSServerPorts extends TestCase {
       // reset conf2 since NameNode modifies it
       conf2.set("fs.default.name", NAME_NODE_HOST + "0");
       // different http port
-      conf2.set("dfs.http.bindAddress", NAME_NODE_HTTP_HOST + "0");
+      conf2.set("dfs.http.address", NAME_NODE_HTTP_HOST + "0");
       started = canStartNameNode(conf2);
       assertTrue(started); // should start now
     } finally {
@@ -163,20 +163,21 @@ public class TestHDFSServerPorts extends TestCase {
       // start data-node on the same port as name-node
       Configuration conf2 = new Configuration(config);
       conf2.set("dfs.data.dir", new File(hdfsDir, "data").getPath());
-      conf2.set("dfs.datanode.bindAddress", config.get("fs.default.name"));
-      conf2.set("dfs.datanode.http.bindAddress", NAME_NODE_HTTP_HOST + "0");
+      conf2.set("dfs.datanode.address", config.get("fs.default.name"));
+      conf2.set("dfs.datanode.http.address", NAME_NODE_HTTP_HOST + "0");
       boolean started = canStartDataNode(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
-      conf2.set("dfs.datanode.bindAddress", NAME_NODE_HOST + "0");
-      conf2.set("dfs.datanode.http.bindAddress", config.get("dfs.http.bindAddress"));
+      conf2.set("dfs.datanode.address", NAME_NODE_HOST + "0");
+      conf2.set("dfs.datanode.http.address", 
+                config.get("dfs.http.address"));
       started = canStartDataNode(conf2);
       assertFalse(started); // should fail
     
       // both ports are different from the name-node ones
-      conf2.set("dfs.datanode.bindAddress", NAME_NODE_HOST + "0");
-      conf2.set("dfs.datanode.http.bindAddress", NAME_NODE_HTTP_HOST + "0");
+      conf2.set("dfs.datanode.address", NAME_NODE_HOST + "0");
+      conf2.set("dfs.datanode.http.address", NAME_NODE_HTTP_HOST + "0");
       started = canStartDataNode(conf2);
       assertTrue(started); // should start now
     } finally {
@@ -194,14 +195,17 @@ public class TestHDFSServerPorts extends TestCase {
 
       // bind http server to the same port as name-node
       Configuration conf2 = new Configuration(config);
-      conf2.set("dfs.secondary.http.bindAddress", config.get("dfs.http.bindAddress"));
-      SecondaryNameNode.LOG.info("= Starting 1 on: " + conf2.get("dfs.secondary.http.bindAddress"));
+      conf2.set("dfs.secondary.http.address", 
+                config.get("dfs.http.address"));
+      SecondaryNameNode.LOG.info("= Starting 1 on: " + 
+                                 conf2.get("dfs.secondary.http.address"));
       boolean started = canStartSecondaryNode(conf2);
       assertFalse(started); // should fail
 
       // bind http server to a different port
-      conf2.set("dfs.secondary.http.bindAddress", NAME_NODE_HTTP_HOST + "0");
-      SecondaryNameNode.LOG.info("= Starting 2 on: " + conf2.get("dfs.secondary.http.bindAddress"));
+      conf2.set("dfs.secondary.http.address", NAME_NODE_HTTP_HOST + "0");
+      SecondaryNameNode.LOG.info("= Starting 2 on: " + 
+                                 conf2.get("dfs.secondary.http.address"));
       started = canStartSecondaryNode(conf2);
       assertTrue(started); // should start now
     } finally {

+ 3 - 3
src/test/org/apache/hadoop/mapred/MiniMRCluster.java

@@ -109,8 +109,8 @@ public class MiniMRCluster {
       this.numDir = numDir;
       localDirs = new String[numDir];
       conf = createJobConf();
-      conf.set("mapred.task.tracker.http.bindAddress", "0.0.0.0:0");
-      conf.set("mapred.task.tracker.report.bindAddress", 
+      conf.set("mapred.task.tracker.http.address", "0.0.0.0:0");
+      conf.set("mapred.task.tracker.report.address", 
                 "127.0.0.1:" + taskTrackerPort);
       File localDirBase = 
         new File(conf.get("mapred.local.dir")).getAbsoluteFile();
@@ -225,7 +225,7 @@ public class MiniMRCluster {
     JobConf result = new JobConf();
     result.set("fs.default.name", namenode);
     result.set("mapred.job.tracker", "localhost:"+jobTrackerPort);
-    result.set("mapred.job.tracker.http.bindAddress", 
+    result.set("mapred.job.tracker.http.address", 
                         "0.0.0.0:" + jobTrackerInfoPort);
     // for debugging have all task output sent to the test output
     JobClient.setTaskOutputFilter(result, JobClient.TaskStatusFilter.ALL);

+ 13 - 13
src/test/org/apache/hadoop/mapred/TestMRServerPorts.java

@@ -41,12 +41,12 @@ public class TestMRServerPorts extends TestCase {
   private JobTracker startJobTracker(JobConf conf) 
   throws IOException {
     conf.set("mapred.job.tracker", "localhost:0");
-    conf.set("mapred.job.tracker.http.bindAddress", "0.0.0.0:0");
+    conf.set("mapred.job.tracker.http.address", "0.0.0.0:0");
     JobTracker jt = null;
     try {
       jt = JobTracker.startTracker(conf);
       conf.set("mapred.job.tracker", "localhost:" + jt.getTrackerPort());
-      conf.set("mapred.job.tracker.http.bindAddress", 
+      conf.set("mapred.job.tracker.http.address", 
                             "0.0.0.0:" + jt.getInfoPort());
     } catch(InterruptedException e) {
       throw new IOException(e.getLocalizedMessage());
@@ -100,21 +100,21 @@ public class TestMRServerPorts extends TestCase {
       // start job tracker on the same port as name-node
       JobConf conf2 = new JobConf(hdfs.getConfig());
       conf2.set("mapred.job.tracker", hdfs.getConfig().get("fs.default.name"));
-      conf2.set("mapred.job.tracker.http.bindAddress",
+      conf2.set("mapred.job.tracker.http.address",
         TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
       boolean started = canStartJobTracker(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
       conf2.set("mapred.job.tracker", TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set("mapred.job.tracker.http.bindAddress",
-        hdfs.getConfig().get("dfs.http.bindAddress"));
+      conf2.set("mapred.job.tracker.http.address",
+        hdfs.getConfig().get("dfs.http.address"));
       started = canStartJobTracker(conf2);
       assertFalse(started); // should fail again
 
       // both ports are different from the name-node ones
       conf2.set("mapred.job.tracker", TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set("mapred.job.tracker.http.bindAddress",
+      conf2.set("mapred.job.tracker.http.address",
         TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
       started = canStartJobTracker(conf2);
       assertTrue(started); // should start now
@@ -137,25 +137,25 @@ public class TestMRServerPorts extends TestCase {
       jt = startJobTracker(conf2);
 
       // start job tracker on the same port as name-node
-      conf2.set("mapred.task.tracker.report.bindAddress",
+      conf2.set("mapred.task.tracker.report.address",
         hdfs.getConfig().get("fs.default.name"));
-      conf2.set("mapred.task.tracker.http.bindAddress",
+      conf2.set("mapred.task.tracker.http.address",
         TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
       boolean started = canStartTaskTracker(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
-      conf2.set("mapred.task.tracker.report.bindAddress",
+      conf2.set("mapred.task.tracker.report.address",
         TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set("mapred.task.tracker.http.bindAddress",
-        hdfs.getConfig().get("dfs.http.bindAddress"));
+      conf2.set("mapred.task.tracker.http.address",
+        hdfs.getConfig().get("dfs.http.address"));
       started = canStartTaskTracker(conf2);
       assertFalse(started); // should fail again
 
       // both ports are different from the name-node ones
-      conf2.set("mapred.task.tracker.report.bindAddress",
+      conf2.set("mapred.task.tracker.report.address",
         TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set("mapred.task.tracker.http.bindAddress",
+      conf2.set("mapred.task.tracker.http.address",
         TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
       started = canStartTaskTracker(conf2);
       assertTrue(started); // should start now