Browse Source

HDFS-9266 Avoid unsafe split and append on fields that might be IPv6 literals

Elliott Clark 10 năm trước cách đây
mục cha
commit
fa64e68fd5
23 tập tin đã thay đổi với 283 bổ sung134 xóa
  1. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  4. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  5. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  6. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  8. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  9. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  10. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
  11. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
  12. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
  14. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/WebImageViewer.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
  16. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
  17. 31 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  18. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  19. 75 71
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  20. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
  21. 27 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java
  22. 70 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
  23. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -860,6 +861,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    */
   public static String getFileName(final InetSocketAddress s,
       final String poolId, final long blockId) {
-    return s.toString() + ":" + poolId + ":" + blockId;
+    return NetUtils.getSocketAddressString(s) + ":" + poolId + ":" + blockId;
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -360,7 +360,7 @@ public class RemoteBlockReader2  implements BlockReader {
    */
   public static String getFileName(final InetSocketAddress s,
       final String poolId, final long blockId) {
-    return s.toString() + ":" + poolId + ":" + blockId;
+    return NetUtils.getSocketAddressString(s) + ":" + poolId + ":" + blockId;
   }
 
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -204,7 +204,7 @@ class JsonUtilClient {
     if (ipAddr == null) {
       String name = getString(m, "name", null);
       if (name != null) {
-        int colonIdx = name.indexOf(':');
+        int colonIdx = name.lastIndexOf(':');
         if (colonIdx > 0) {
           ipAddr = name.substring(0, colonIdx);
           xferPort = Integer.parseInt(name.substring(colonIdx +1));

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
+import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -51,12 +52,12 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StopWatch;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.net.InetAddresses;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -672,8 +673,8 @@ public class IPCLoggerChannel implements AsyncLogger {
 
   @Override
   public String toString() {
-    return InetAddresses.toAddrString(addr.getAddress()) + ':' +
-        addr.getPort();
+    final InetAddress ip = addr.getAddress();
+    return NetUtils.getHostPortString(addr);
   }
 
   @Override

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.http.conn.util.InetAddressUtils;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -1193,7 +1194,13 @@ public class DatanodeManager {
     DatanodeID dnId;
     String hostStr;
     int port;
-    int idx = hostLine.indexOf(':');
+    int idx;
+
+    if (InetAddressUtils.isIPv6StdAddress(hostLine)) {
+      idx = -1;
+    } else {
+      idx = hostLine.lastIndexOf(':');
+    }
 
     if (-1 == idx) {
       hostStr = hostLine;

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -77,10 +77,20 @@ public class BlockPoolSliceStorage extends Storage {
    *      progress. Do not delete the 'previous' directory.
    */
   static final String ROLLING_UPGRADE_MARKER_FILE = "RollingUpgradeInProgress";
+  private static final String BLOCK_POOL_ID_IPV4_PATTERN_BASE =
+      "\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}";
+
+  // Because we don't support ":" in path BlockPoolID on IPv6 boxes we replace
+  // ":" with ".".
+  // Also format of IPv6 is less fixed so we surround it with square brackets
+  // and just check that match
+  private static final String BLOCK_POOL_ID_IPV6_PATTERN_BASE =
+      Pattern.quote("[") + "(?:.*)" + Pattern.quote("]");
 
   private static final String BLOCK_POOL_ID_PATTERN_BASE =
       Pattern.quote(File.separator) +
-      "BP-\\d+-\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}-\\d+" +
+      "BP-\\d+-(?:" + BLOCK_POOL_ID_IPV4_PATTERN_BASE + "|" +
+      BLOCK_POOL_ID_IPV6_PATTERN_BASE + ")-\\d+" +
       Pattern.quote(File.separator);
 
   private static final Pattern BLOCK_POOL_PATH_PATTERN = Pattern.compile(

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -137,7 +137,7 @@ class DataXceiver extends Receiver implements Runnable {
     this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(datanode.getConf());
     this.smallBufferSize = DFSUtilClient.getSmallBufferSize(datanode.getConf());
     remoteAddress = peer.getRemoteAddressString();
-    final int colonIdx = remoteAddress.indexOf(':');
+    final int colonIdx = remoteAddress.lastIndexOf(':');
     remoteAddressWithoutPort =
         (colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx);
     localAddress = peer.getLocalAddressString();

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -101,10 +101,9 @@ class Checkpointer extends Daemon {
     checkpointConf = new CheckpointConf(conf);
 
     // Pull out exact http address for posting url to avoid ip aliasing issues
-    String fullInfoAddr = conf.get(DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, 
+    String fullInfoAddr = conf.get(DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
                                    DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT);
-    infoBindAddress = fullInfoAddr.substring(0, fullInfoAddr.indexOf(":"));
-
+    infoBindAddress = fullInfoAddr.substring(0, fullInfoAddr.lastIndexOf(":"));
     LOG.info("Checkpoint Period : " +
              checkpointConf.getPeriod() + " secs " +
              "(" + checkpointConf.getPeriod()/60 + " min)");

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.util.PersistentLongFile;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.util.Time;
+import org.apache.http.conn.util.InetAddressUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -964,11 +965,15 @@ public class NNStorage extends Storage implements Closeable,
     String ip = "unknownIP";
     try {
       ip = DNS.getDefaultIP("default");
+      if (InetAddressUtils.isIPv6StdAddress(ip)) {
+        // HDFS doesn't support ":" in path, replace it with "."
+        ip = "[" + ip.replaceAll(":", ".") + "]";
+      }
     } catch (UnknownHostException e) {
       LOG.warn("Could not find ip address of \"default\" inteface.");
       throw e;
     }
-    
+
     int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
     String bpid = "BP-" + rand + "-"+ ip + "-" + Time.now();
     return bpid;
@@ -986,14 +991,14 @@ public class NNStorage extends Storage implements Closeable,
       throw new InconsistentFSStateException(storage, "file "
           + Storage.STORAGE_FILE_VERSION + " has no block pool Id.");
     }
-    
+
     if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) {
       throw new InconsistentFSStateException(storage,
           "Unexepcted blockpoolID " + bpid + " . Expected " + blockpoolID);
     }
     setBlockPoolID(bpid);
   }
-  
+
   public String getBlockPoolID() {
     return blockpoolID;
   }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -155,7 +156,8 @@ public class ConfiguredFailoverProxyProvider<T> extends
         throw new RuntimeException(e);
       }
     }
-    return new ProxyInfo<T>(current.namenode, current.address.toString());
+    return new ProxyInfo<T>(current.namenode,
+        NetUtils.getSocketAddressString(current.address));
   }
 
   @Override

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.common.base.Preconditions;
@@ -94,7 +95,7 @@ public class IPFailoverProxyProvider<T> extends
         InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
         nnProxyInfo = new ProxyInfo<T>(NameNodeProxies.createNonHAProxy(
             conf, nnAddr, xface, UserGroupInformation.getCurrentUser(), 
-            false).getProxy(), nnAddr.toString());
+            false).getProxy(), NetUtils.getSocketAddressString(nnAddr));
       } catch (IOException ioe) {
         throw new RuntimeException(ioe);
       }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -93,6 +93,7 @@ import org.apache.hadoop.util.StringUtils;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.collect.Lists;
+import com.google.common.net.HostAndPort;
 import com.sun.jersey.spi.container.ResourceFilters;
 
 /** Web-hdfs NameNode implementation. */
@@ -171,17 +172,17 @@ public class NamenodeWebHdfsMethods {
       throw new IOException("Namesystem has not been intialized yet.");
     }
     final BlockManager bm = fsn.getBlockManager();
-    
+
     HashSet<Node> excludes = new HashSet<Node>();
     if (excludeDatanodes != null) {
-      for (String host : StringUtils
+      for (String hostAndPort : StringUtils
           .getTrimmedStringCollection(excludeDatanodes)) {
-        int idx = host.indexOf(":");
-        if (idx != -1) {          
+        HostAndPort hp = HostAndPort.fromString(hostAndPort);
+        if (hp.hasPort()) {
           excludes.add(bm.getDatanodeManager().getDatanodeByXferAddr(
-              host.substring(0, idx), Integer.parseInt(host.substring(idx + 1))));
+              hp.getHostText(), hp.getPort()));
         } else {
-          excludes.add(bm.getDatanodeManager().getDatanodeByHost(host));
+          excludes.add(bm.getDatanodeManager().getDatanodeByHost(hostAndPort));
         }
       }
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -228,7 +229,7 @@ public class GetConf extends Configured implements Tool {
       if (!cnnlist.isEmpty()) {
         for (ConfiguredNNAddress cnn : cnnlist) {
           InetSocketAddress rpc = cnn.getAddress();
-          tool.printOut(rpc.getHostName()+":"+rpc.getPort());
+          tool.printOut(NetUtils.getHostPortString(rpc));
         }
         return 0;
       }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/WebImageViewer.java

@@ -34,6 +34,7 @@ import io.netty.handler.codec.string.StringEncoder;
 import io.netty.util.concurrent.GlobalEventExecutor;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -104,7 +105,9 @@ public class WebImageViewer implements Closeable {
     allChannels.add(channel);
 
     address = (InetSocketAddress) channel.localAddress();
-    LOG.info("WebImageViewer started. Listening on " + address.toString() + ". Press Ctrl+C to stop the viewer.");
+    LOG.info("WebImageViewer started. Listening on " + NetUtils
+        .getSocketAddressString(address) +
+        ". Press Ctrl+C to stop the viewer.");
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -181,7 +181,7 @@ public class BlockReaderTestUtil {
     return new BlockReaderFactory(fs.getClient().getConf()).
       setInetSocketAddress(targetAddr).
       setBlock(block).
-      setFileName(targetAddr.toString()+ ":" + block.getBlockId()).
+      setFileName(NetUtils.getSocketAddressString(targetAddr)+ ":" + block.getBlockId()).
       setBlockToken(testBlock.getBlockToken()).
       setStartOffset(offset).
       setLength(lenToRead).

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.net.NetUtils;
 import org.junit.Test;
 
 
@@ -55,7 +56,7 @@ public class TestDFSAddressConfig {
     ArrayList<DataNode> dns = cluster.getDataNodes();
     DataNode dn = dns.get(0);
 
-    String selfSocketAddr = dn.getXferAddress().toString();
+    String selfSocketAddr = NetUtils.getSocketAddressString(dn.getXferAddress());
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
     assertTrue(selfSocketAddr.contains("/127.0.0.1:"));
 
@@ -80,7 +81,7 @@ public class TestDFSAddressConfig {
     dns = cluster.getDataNodes();
     dn = dns.get(0);
 
-    selfSocketAddr = dn.getXferAddress().toString();
+    selfSocketAddr = NetUtils.getSocketAddressString(dn.getXferAddress());
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
     // assert that default self socket address is 127.0.0.1
     assertTrue(selfSocketAddr.contains("/127.0.0.1:"));
@@ -105,10 +106,11 @@ public class TestDFSAddressConfig {
     dns = cluster.getDataNodes();
     dn = dns.get(0);
 
-    selfSocketAddr = dn.getXferAddress().toString();
+    selfSocketAddr = NetUtils.getSocketAddressString(dn.getXferAddress());
     System.out.println("DN Self Socket Addr == " + selfSocketAddr);
     // assert that default self socket address is 0.0.0.0
-    assertTrue(selfSocketAddr.contains("/0.0.0.0:"));
+    assertTrue(selfSocketAddr.contains("/0.0.0.0:") ||
+        selfSocketAddr.contains("/[0:0:0:0:0:0:0:0]:"));
 
     cluster.shutdown();
   }

+ 31 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -502,26 +502,26 @@ public class TestDFSUtil {
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"),
         NS2_NN1_HOST);
     conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"),
+            DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"),
         NS2_NN2_HOST);
-    
+
     Map<String, Map<String, InetSocketAddress>> map =
       DFSUtil.getHaNnRpcAddresses(conf);
 
     assertTrue(HAUtil.isHAEnabled(conf, "ns1"));
     assertTrue(HAUtil.isHAEnabled(conf, "ns2"));
     assertFalse(HAUtil.isHAEnabled(conf, "ns3"));
-    
-    assertEquals(NS1_NN1_HOST, map.get("ns1").get("ns1-nn1").toString());
-    assertEquals(NS1_NN2_HOST, map.get("ns1").get("ns1-nn2").toString());
-    assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString());
-    assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString());
-    
-    assertEquals(NS1_NN1_HOST, 
+
+    assertEquals(NS1_NN1_HOST, NetUtils.getHostPortString(map.get("ns1").get("ns1-nn1")));
+    assertEquals(NS1_NN2_HOST, NetUtils.getHostPortString(map.get("ns1").get("ns1-nn2")));
+    assertEquals(NS2_NN1_HOST, NetUtils.getHostPortString(map.get("ns2").get("ns2-nn1")));
+    assertEquals(NS2_NN2_HOST, NetUtils.getHostPortString(map.get("ns2").get("ns2-nn2")));
+
+    assertEquals(NS1_NN1_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1"));
-    assertEquals(NS1_NN2_HOST, 
+    assertEquals(NS1_NN2_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2"));
-    assertEquals(NS2_NN1_HOST, 
+    assertEquals(NS2_NN1_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1"));
 
     // No nameservice was given and we can't determine which service addr
@@ -589,8 +589,26 @@ public class TestDFSUtil {
     Map<String, Map<String, InetSocketAddress>> map =
         DFSUtilClient.getHaNnWebHdfsAddresses(conf, "webhdfs");
 
-    assertEquals(NS1_NN1_ADDR, map.get("ns1").get("nn1").toString());
-    assertEquals(NS1_NN2_ADDR, map.get("ns1").get("nn2").toString());
+    assertEquals(NS1_NN1_ADDR, NetUtils.getHostPortString(map.get("ns1").get("nn1")));
+    assertEquals(NS1_NN2_ADDR, NetUtils.getHostPortString(map.get(
+        "ns1").get("nn2")));
+  }
+
+  @Test
+  public void testIPv6GetHaNnHttpAddresses() throws IOException {
+    final String LOGICAL_HOST_NAME = "ns1";
+    final String NS1_NN1_ADDR      = "[0:0:0:0:0:b00c:c0a8:12a]:8020";
+    final String NS1_NN2_ADDR      = "[::face:a0b:182a]:8020";
+
+    Configuration conf = createWebHDFSHAConfiguration(LOGICAL_HOST_NAME, NS1_NN1_ADDR, NS1_NN2_ADDR);
+
+    Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtilClient.getHaNnWebHdfsAddresses(conf, "webhdfs");
+
+    assertEquals(NS1_NN1_ADDR, NetUtils.getHostPortString(map.get("ns1").get("nn1")));
+    assertEquals(NS1_NN2_ADDR.replace("::", "0:0:0:0:0:"),
+        NetUtils.getHostPortString(map.get(
+        "ns1").get("nn2")));
   }
 
   private static Configuration createWebHDFSHAConfiguration(String logicalHostName, String nnaddr1, String nnaddr2) {

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -424,7 +425,8 @@ public class TestFileAppend{
 
       // stop one datanode
       DataNodeProperties dnProp = cluster.stopDataNode(0);
-      String dnAddress = dnProp.datanode.getXferAddress().toString();
+      String dnAddress = NetUtils.getSocketAddressString(
+          dnProp.datanode.getXferAddress());
       if (dnAddress.startsWith("/")) {
         dnAddress = dnAddress.substring(1);
       }
@@ -478,7 +480,8 @@ public class TestFileAppend{
 
       // stop one datanode
       DataNodeProperties dnProp = cluster.stopDataNode(0);
-      String dnAddress = dnProp.datanode.getXferAddress().toString();
+      String dnAddress = NetUtils
+          .getSocketAddressString(dnProp.datanode.getXferAddress());
       if (dnAddress.startsWith("/")) {
         dnAddress = dnAddress.substring(1);
       }

+ 75 - 71
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -86,6 +86,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -93,6 +95,7 @@ import org.junit.Test;
  * This class tests various cases during file creation.
  */
 public class TestFileCreation {
+  public static final Log LOG = LogFactory.getLog(TestFileCreation.class);
   static final String DIR = "/" + TestFileCreation.class.getSimpleName() + "/";
 
   {
@@ -108,7 +111,7 @@ public class TestFileCreation {
   static final int numBlocks = 2;
   static final int fileSize = numBlocks * blockSize + 1;
   boolean simulatedStorage = false;
-  
+
   private static final String[] NON_CANONICAL_PATHS = new String[] {
     "//foo",
     "///foo2",
@@ -121,7 +124,7 @@ public class TestFileCreation {
   // creates a file but does not close it
   public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
-    System.out.println("createFile: Created " + name + " with " + repl + " replica.");
+    LOG.info("createFile: Created " + name + " with " + repl + " replica.");
     FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
         .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
         (short) repl, blockSize);
@@ -194,8 +197,8 @@ public class TestFileCreation {
   public void testFileCreationSetLocalInterface() throws IOException {
     assumeTrue(System.getProperty("os.name").startsWith("Linux"));
 
-    // The mini cluster listens on the loopback so we can use it here
-    checkFileCreation("lo", false);
+    // Use wildcard address to force interface to be used
+    checkFileCreation("0.0.0.0", false);
 
     try {
       checkFileCreation("bogus-interface", false);
@@ -212,6 +215,7 @@ public class TestFileCreation {
    */
   public void checkFileCreation(String netIf, boolean useDnHostname)
       throws IOException {
+
     Configuration conf = new HdfsConfiguration();
     if (netIf != null) {
       conf.set(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES, netIf);
@@ -237,9 +241,9 @@ public class TestFileCreation {
       // check that / exists
       //
       Path path = new Path("/");
-      System.out.println("Path : \"" + path.toString() + "\"");
-      System.out.println(fs.getFileStatus(path).isDirectory()); 
-      assertTrue("/ should be a directory", 
+      LOG.info("Path : \"" + path.toString() + "\"");
+      LOG.info(fs.getFileStatus(path).isDirectory());
+      assertTrue("/ should be a directory",
                  fs.getFileStatus(path).isDirectory());
 
       //
@@ -247,7 +251,7 @@ public class TestFileCreation {
       //
       Path dir1 = new Path("/test_dir");
       fs.mkdirs(dir1);
-      System.out.println("createFile: Creating " + dir1.getName() + 
+      LOG.info("createFile: Creating " + dir1.getName() +
         " for overwrite of existing directory.");
       try {
         fs.create(dir1, true); // Create path, overwrite=true
@@ -268,9 +272,9 @@ public class TestFileCreation {
       FSDataOutputStream stm = createFile(fs, file1, 1);
 
       // verify that file exists in FS namespace
-      assertTrue(file1 + " should be a file", 
+      assertTrue(file1 + " should be a file",
                  fs.getFileStatus(file1).isFile());
-      System.out.println("Path : \"" + file1 + "\"");
+      LOG.info("Path : \"" + file1 + "\"");
 
       // write to file
       writeFile(stm);
@@ -282,13 +286,13 @@ public class TestFileCreation {
       assertTrue(file1 + " should be of size " + fileSize +
                  " but found to be of size " + len, 
                   len == fileSize);
-      
+
       // verify the disk space the file occupied
       long diskSpace = dfs.getContentSummary(file1.getParent()).getLength();
       assertEquals(file1 + " should take " + fileSize + " bytes disk space " +
           "but found to take " + diskSpace + " bytes", fileSize, diskSpace);
-      
-      // Check storage usage 
+
+      // Check storage usage
       // can't check capacities for real storage since the OS file system may be changing under us.
       if (simulatedStorage) {
         DataNode dn = cluster.getDataNodes().get(0);
@@ -325,7 +329,7 @@ public class TestFileCreation {
       FSDataOutputStream stm1 = createFile(fs, file1, 1);
       FSDataOutputStream stm2 = createFile(fs, file2, 1);
       FSDataOutputStream stm3 = createFile(localfs, file3, 1);
-      System.out.println("DeleteOnExit: Created files.");
+      LOG.info("DeleteOnExit: Created files.");
 
       // write to files and close. Purposely, do not close file2.
       writeFile(stm1);
@@ -356,7 +360,7 @@ public class TestFileCreation {
                  !fs.exists(file2));
       assertTrue(file3 + " still exists inspite of deletOnExit set.",
                  !localfs.exists(file3));
-      System.out.println("DeleteOnExit successful.");
+      LOG.info("DeleteOnExit successful.");
 
     } finally {
       IOUtils.closeStream(fs);
@@ -453,7 +457,7 @@ public class TestFileCreation {
       // verify that file exists in FS namespace
       assertTrue(file1 + " should be a file", 
                  fs.getFileStatus(file1).isFile());
-      System.out.println("Path : \"" + file1 + "\"");
+      LOG.info("Path : \"" + file1 + "\"");
 
       // kill the datanode
       cluster.shutdownDataNodes();
@@ -465,7 +469,7 @@ public class TestFileCreation {
         if (info.length == 0) {
           break;
         }
-        System.out.println("testFileCreationError1: waiting for datanode " +
+        LOG.info("testFileCreationError1: waiting for datanode " +
                            " to die.");
         try {
           Thread.sleep(1000);
@@ -480,14 +484,14 @@ public class TestFileCreation {
         stm.write(buffer);
         stm.close();
       } catch (Exception e) {
-        System.out.println("Encountered expected exception");
+        LOG.info("Encountered expected exception");
       }
 
       // verify that no blocks are associated with this file
       // bad block allocations were cleaned up earlier.
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
-      System.out.println("locations = " + locations.locatedBlockCount());
+      LOG.info("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up",
                  locations.locatedBlockCount() == 0);
     } finally {
@@ -503,7 +507,7 @@ public class TestFileCreation {
   @Test
   public void testFileCreationError2() throws IOException {
     long leasePeriod = 1000;
-    System.out.println("testFileCreationError2 start");
+    LOG.info("testFileCreationError2 start");
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
@@ -522,24 +526,24 @@ public class TestFileCreation {
       //
       Path file1 = new Path("/filestatus.dat");
       createFile(dfs, file1, 1);
-      System.out.println("testFileCreationError2: "
+      LOG.info("testFileCreationError2: "
                          + "Created file filestatus.dat with one replicas.");
 
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
-      System.out.println("testFileCreationError2: "
+      LOG.info("testFileCreationError2: "
           + "The file has " + locations.locatedBlockCount() + " blocks.");
 
       // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
           client.clientName, null, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
-      System.out.println("testFileCreationError2: "
+      LOG.info("testFileCreationError2: "
           + "Added block " + location.getBlock());
 
       locations = client.getNamenode().getBlockLocations(file1.toString(), 
                                                     0, Long.MAX_VALUE);
       int count = locations.locatedBlockCount();
-      System.out.println("testFileCreationError2: "
+      LOG.info("testFileCreationError2: "
           + "The file now has " + count + " blocks.");
       
       // set the soft and hard limit to be 1 second so that the
@@ -555,10 +559,10 @@ public class TestFileCreation {
       // verify that the last block was synchronized.
       locations = client.getNamenode().getBlockLocations(file1.toString(), 
                                                     0, Long.MAX_VALUE);
-      System.out.println("testFileCreationError2: "
+      LOG.info("testFileCreationError2: "
           + "locations = " + locations.locatedBlockCount());
       assertEquals(0, locations.locatedBlockCount());
-      System.out.println("testFileCreationError2 successful");
+      LOG.info("testFileCreationError2 successful");
     } finally {
       IOUtils.closeStream(dfs);
       cluster.shutdown();
@@ -568,7 +572,7 @@ public class TestFileCreation {
   /** test addBlock(..) when replication<min and excludeNodes==null. */
   @Test
   public void testFileCreationError3() throws IOException {
-    System.out.println("testFileCreationError3 start");
+    LOG.info("testFileCreationError3 start");
     Configuration conf = new HdfsConfiguration();
     // create cluster
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
@@ -589,7 +593,7 @@ public class TestFileCreation {
         FileSystem.LOG.info("GOOD!", ioe);
       }
 
-      System.out.println("testFileCreationError3 successful");
+      LOG.info("testFileCreationError3 successful");
     } finally {
       IOUtils.closeStream(dfs);
       cluster.shutdown();
@@ -622,7 +626,7 @@ public class TestFileCreation {
       // create a new file.
       Path file1 = new Path("/filestatus.dat");
       HdfsDataOutputStream stm = create(fs, file1, 1);
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Created file " + file1);
       assertEquals(file1 + " should be replicated to 1 datanode.", 1,
           stm.getCurrentBlockReplication());
@@ -636,7 +640,7 @@ public class TestFileCreation {
       // rename file wile keeping it open.
       Path fileRenamed = new Path("/filestatusRenamed.dat");
       fs.rename(file1, fileRenamed);
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Renamed file " + file1 + " to " +
                          fileRenamed);
       file1 = fileRenamed;
@@ -645,7 +649,7 @@ public class TestFileCreation {
       //
       Path file2 = new Path("/filestatus2.dat");
       FSDataOutputStream stm2 = createFile(fs, file2, 1);
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Created file " + file2);
 
       // create yet another new file with full path name. 
@@ -653,21 +657,21 @@ public class TestFileCreation {
       //
       Path file3 = new Path("/user/home/fullpath.dat");
       FSDataOutputStream stm3 = createFile(fs, file3, 1);
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Created file " + file3);
       Path file4 = new Path("/user/home/fullpath4.dat");
       FSDataOutputStream stm4 = createFile(fs, file4, 1);
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Created file " + file4);
 
       fs.mkdirs(new Path("/bin"));
       fs.rename(new Path("/user/home"), new Path("/bin"));
       Path file3new = new Path("/bin/home/fullpath.dat");
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Renamed file " + file3 + " to " +
                          file3new);
       Path file4new = new Path("/bin/home/fullpath4.dat");
-      System.out.println("testFileCreationNamenodeRestart: "
+      LOG.info("testFileCreationNamenodeRestart: "
                          + "Renamed file " + file4 + " to " +
                          file4new);
 
@@ -727,14 +731,14 @@ public class TestFileCreation {
       DFSClient client = fs.dfs;
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
-      System.out.println("locations = " + locations.locatedBlockCount());
+      LOG.info("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up for file " + file1,
                  locations.locatedBlockCount() == 3);
 
       // verify filestatus2.dat
       locations = client.getNamenode().getBlockLocations(
                                   file2.toString(), 0, Long.MAX_VALUE);
-      System.out.println("locations = " + locations.locatedBlockCount());
+      LOG.info("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up for file " + file2,
                  locations.locatedBlockCount() == 1);
     } finally {
@@ -749,7 +753,7 @@ public class TestFileCreation {
   @Test
   public void testDFSClientDeath() throws IOException, InterruptedException {
     Configuration conf = new HdfsConfiguration();
-    System.out.println("Testing adbornal client death.");
+    LOG.info("Testing adbornal client death.");
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }
@@ -763,7 +767,7 @@ public class TestFileCreation {
       //
       Path file1 = new Path("/clienttest.dat");
       FSDataOutputStream stm = createFile(fs, file1, 1);
-      System.out.println("Created file clienttest.dat");
+      LOG.info("Created file clienttest.dat");
 
       // write to file
       writeFile(stm);
@@ -779,7 +783,7 @@ public class TestFileCreation {
       cluster.shutdown();
     }
   }
-  
+
   /**
    * Test file creation using createNonRecursive().
    */
@@ -861,7 +865,7 @@ public class TestFileCreation {
   static IOException createNonRecursive(FileSystem fs, Path name,
       int repl, EnumSet<CreateFlag> flag) throws IOException {
     try {
-      System.out.println("createNonRecursive: Attempting to create " + name +
+      LOG.info("createNonRecursive: Attempting to create " + name +
           " with " + repl + " replica.");
       int bufferSize = fs.getConf()
           .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
@@ -895,9 +899,9 @@ public class TestFileCreation {
 
     try {
       FileSystem fs = cluster.getFileSystem();
-      
+
       Path[] p = {new Path("/foo"), new Path("/bar")};
-      
+
       //write 2 files at the same time
       FSDataOutputStream[] out = {fs.create(p[0]), fs.create(p[1])};
       int i = 0;
@@ -929,9 +933,9 @@ public class TestFileCreation {
 
     try {
       FileSystem fs = cluster.getFileSystem();
-      
+
       Path[] p = {new Path("/foo"), new Path("/bar")};
-      
+
       //write 2 files at the same time
       FSDataOutputStream[] out = {fs.create(p[0]), fs.create(p[1])};
       int i = 0;
@@ -959,7 +963,7 @@ public class TestFileCreation {
    */
   @Test
   public void testLeaseExpireHardLimit() throws Exception {
-    System.out.println("testLeaseExpireHardLimit start");
+    LOG.info("testLeaseExpireHardLimit start");
     final long leasePeriod = 1000;
     final int DATANODE_NUM = 3;
 
@@ -1004,20 +1008,20 @@ public class TestFileCreation {
           successcount++;
         }
       }
-      System.out.println("successcount=" + successcount);
-      assertTrue(successcount > 0); 
+      LOG.info("successcount=" + successcount);
+      assertTrue(successcount > 0);
     } finally {
       IOUtils.closeStream(dfs);
       cluster.shutdown();
     }
 
-    System.out.println("testLeaseExpireHardLimit successful");
+    LOG.info("testLeaseExpireHardLimit successful");
   }
 
   // test closing file system before all file handles are closed.
   @Test
   public void testFsClose() throws Exception {
-    System.out.println("test file system close start");
+    LOG.info("test file system close start");
     final int DATANODE_NUM = 3;
 
     Configuration conf = new HdfsConfiguration();
@@ -1038,7 +1042,7 @@ public class TestFileCreation {
       // close file system without closing file
       dfs.close();
     } finally {
-      System.out.println("testFsClose successful");
+      LOG.info("testFsClose successful");
       cluster.shutdown();
     }
   }
@@ -1046,7 +1050,7 @@ public class TestFileCreation {
   // test closing file after cluster is shutdown
   @Test
   public void testFsCloseAfterClusterShutdown() throws IOException {
-    System.out.println("test testFsCloseAfterClusterShutdown start");
+    LOG.info("test testFsCloseAfterClusterShutdown start");
     final int DATANODE_NUM = 3;
 
     Configuration conf = new HdfsConfiguration();
@@ -1077,13 +1081,13 @@ public class TestFileCreation {
       boolean hasException = false;
       try {
         out.close();
-        System.out.println("testFsCloseAfterClusterShutdown: Error here");
+        LOG.info("testFsCloseAfterClusterShutdown: Error here");
       } catch (IOException e) {
         hasException = true;
       }
       assertTrue("Failed to close file after cluster shutdown", hasException);
     } finally {
-      System.out.println("testFsCloseAfterClusterShutdown successful");
+      LOG.info("testFsCloseAfterClusterShutdown successful");
       if (cluster != null) {
         cluster.shutdown();
       }
@@ -1102,7 +1106,7 @@ public class TestFileCreation {
   public void testCreateNonCanonicalPathAndRestartRpc() throws Exception {
     doCreateTest(CreationMethod.DIRECT_NN_RPC);
   }
-  
+
   /**
    * Another regression test for HDFS-3626. This one creates files using
    * a Path instantiated from a string object.
@@ -1122,7 +1126,7 @@ public class TestFileCreation {
       throws Exception {
     doCreateTest(CreationMethod.PATH_FROM_URI);
   }
-  
+
   private static enum CreationMethod {
     DIRECT_NN_RPC,
     PATH_FROM_URI,
@@ -1137,7 +1141,7 @@ public class TestFileCreation {
       NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
 
       for (String pathStr : NON_CANONICAL_PATHS) {
-        System.out.println("Creating " + pathStr + " by " + method);
+        LOG.info("Creating " + pathStr + " by " + method);
         switch (method) {
         case DIRECT_NN_RPC:
           try {
@@ -1152,7 +1156,7 @@ public class TestFileCreation {
             // So, we expect all of them to fail. 
           }
           break;
-          
+
         case PATH_FROM_URI:
         case PATH_FROM_STRING:
           // Unlike the above direct-to-NN case, we expect these to succeed,
@@ -1170,7 +1174,7 @@ public class TestFileCreation {
           throw new AssertionError("bad method: " + method);
         }
       }
-      
+
       cluster.restartNameNode();
 
     } finally {
@@ -1227,13 +1231,13 @@ public class TestFileCreation {
       dfs.mkdirs(new Path("/foo/dir"));
       String file = "/foo/dir/file";
       Path filePath = new Path(file);
-      
+
       // Case 1: Create file with overwrite, check the blocks of old file
       // are cleaned after creating with overwrite
       NameNode nn = cluster.getNameNode();
       FSNamesystem fsn = NameNodeAdapter.getNamesystem(nn);
       BlockManager bm = fsn.getBlockManager();
-      
+
       FSDataOutputStream out = dfs.create(filePath);
       byte[] oldData = AppendTestUtil.randomBytes(seed, fileSize);
       try {
@@ -1241,11 +1245,11 @@ public class TestFileCreation {
       } finally {
         out.close();
       }
-      
+
       LocatedBlocks oldBlocks = NameNodeAdapter.getBlockLocations(
           nn, file, 0, fileSize);
       assertBlocks(bm, oldBlocks, true);
-      
+
       out = dfs.create(filePath, true);
       byte[] newData = AppendTestUtil.randomBytes(seed, fileSize);
       try {
@@ -1254,12 +1258,12 @@ public class TestFileCreation {
         out.close();
       }
       dfs.deleteOnExit(filePath);
-      
+
       LocatedBlocks newBlocks = NameNodeAdapter.getBlockLocations(
           nn, file, 0, fileSize);
       assertBlocks(bm, newBlocks, true);
       assertBlocks(bm, oldBlocks, false);
-      
+
       FSDataInputStream in = dfs.open(filePath);
       byte[] result = null;
       try {
@@ -1268,7 +1272,7 @@ public class TestFileCreation {
         in.close();
       }
       Assert.assertArrayEquals(newData, result);
-      
+
       // Case 2: Restart NN, check the file
       cluster.restartNameNode();
       nn = cluster.getNameNode();
@@ -1279,13 +1283,13 @@ public class TestFileCreation {
         in.close();
       }
       Assert.assertArrayEquals(newData, result);
-      
+
       // Case 3: Save new checkpoint and restart NN, check the file
       NameNodeAdapter.enterSafeMode(nn, false);
       NameNodeAdapter.saveNamespace(nn);
       cluster.restartNameNode();
       nn = cluster.getNameNode();
-      
+
       in = dfs.open(filePath);
       try {
         result = readAll(in);
@@ -1302,8 +1306,8 @@ public class TestFileCreation {
       }
     }
   }
-  
-  private void assertBlocks(BlockManager bm, LocatedBlocks lbs, 
+
+  private void assertBlocks(BlockManager bm, LocatedBlocks lbs,
       boolean exist) {
     for (LocatedBlock locatedBlock : lbs.getLocatedBlocks()) {
       if (exist) {
@@ -1315,7 +1319,7 @@ public class TestFileCreation {
       }
     }
   }
-  
+
   private byte[] readAll(FSDataInputStream in) throws IOException {
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     byte[] buffer = new byte[1024];

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java

@@ -902,14 +902,14 @@ public class TestQuorumJournalManager {
     GenericTestUtils.assertGlobEquals(paxosDir, "\\d+",
         "3");
   }
-  
+
   @Test
   public void testToString() throws Exception {
     GenericTestUtils.assertMatches(
         qjm.toString(),
-        "QJM to \\[127.0.0.1:\\d+, 127.0.0.1:\\d+, 127.0.0.1:\\d+\\]");
+        "QJM to \\[localhost:\\d+, localhost:\\d+, localhost:\\d+\\]");
   }
-  
+
   @Test
   public void testSelectInputStreamsNotOnBoundary() throws Exception {
     final int txIdsPerSegment = 10; 

+ 27 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.junit.Test;
 
 import java.io.File;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.Random;
 import java.util.UUID;
 
@@ -53,13 +55,37 @@ public class TestBlockPoolSliceStorage {
     }
   }
 
-  private String makeRandomIpAddress() {
+  private String makeRandomIpv4Address() {
     return rand.nextInt(256) + "." +
            rand.nextInt(256) + "." +
            rand.nextInt(256) + "." +
            rand.nextInt(256);
   }
 
+  private String makeRandomIpv6Address() {
+    byte[] bytes = new byte[16];
+    rand.nextBytes(bytes);
+    InetAddress adr = null;
+    try {
+      adr = InetAddress.getByAddress("unused", bytes);
+    } catch (UnknownHostException uhe) {
+      // Should never happen
+      LOG.error(uhe);
+      assertThat(true, is(false));
+    }
+    String addrString = adr.getHostAddress().replaceAll(":", ".");
+
+    return "[" + addrString + "]";
+  }
+
+  private String makeRandomIpAddress() {
+    if (rand.nextBoolean()) {
+      return makeRandomIpv4Address();
+    } else {
+      return makeRandomIpv6Address();
+    }
+  }
+
   private String makeRandomBlockpoolId() {
     return "BP-" + rand.nextInt(Integer.MAX_VALUE) +
            "-" + makeRandomIpAddress() +

+ 70 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java

@@ -20,8 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.AnyOf.anyOf;
 import static org.hamcrest.core.IsNot.not;
 
+import org.apache.hadoop.net.NetUtils;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FileUtil;
@@ -29,6 +31,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 
+import java.net.InetAddress;
+import java.net.Inet6Address;
 import java.io.File;
 import java.io.IOException;
 
@@ -53,6 +57,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 public class TestNameNodeRespectsBindHostKeys {
   public static final Log LOG = LogFactory.getLog(TestNameNodeRespectsBindHostKeys.class);
   private static final String WILDCARD_ADDRESS = "0.0.0.0";
+  private static final String IPv6_WILDCARD_ADDRESS = "0:0:0:0:0:0:0:0";
   private static final String LOCALHOST_SERVER_ADDRESS = "127.0.0.1:0";
 
   private static String getRpcServerAddress(MiniDFSCluster cluster) {
@@ -69,9 +74,9 @@ public class TestNameNodeRespectsBindHostKeys {
   public void testRpcBindHostKey() throws IOException {
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
-    
+
     LOG.info("Testing without " + DFS_NAMENODE_RPC_BIND_HOST_KEY);
-    
+
     // NN should not bind the wildcard address by default.
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
@@ -87,7 +92,7 @@ public class TestNameNodeRespectsBindHostKeys {
     }
 
     LOG.info("Testing with " + DFS_NAMENODE_RPC_BIND_HOST_KEY);
-    
+
     // Tell NN to bind the wildcard address.
     conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY, WILDCARD_ADDRESS);
 
@@ -97,12 +102,37 @@ public class TestNameNodeRespectsBindHostKeys {
       cluster.waitActive();
       String address = getRpcServerAddress(cluster);
       assertThat("Bind address " + address + " is not wildcard.",
-                 address, is("/" + WILDCARD_ADDRESS));
+                 address, anyOf(
+                   is("/" + WILDCARD_ADDRESS),
+                   is("/" + IPv6_WILDCARD_ADDRESS)));
     } finally {
       if (cluster != null) {
         cluster.shutdown();
       }
-    }    
+    }
+
+    InetAddress localAddr = InetAddress.getLocalHost();
+    if (localAddr instanceof Inet6Address) {
+      // Tell NN to bind the IPv6 wildcard address.
+      conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY, IPv6_WILDCARD_ADDRESS);
+
+      // Verify that NN binds wildcard address now.
+      try {
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+        cluster.waitActive();
+        String address = getRpcServerAddress(cluster);
+        assertThat("Bind address " + address + " is not wildcard.",
+                   address, anyOf(
+                     is("/" + WILDCARD_ADDRESS),
+                     is("/" + IPv6_WILDCARD_ADDRESS)));
+      } finally {
+        if (cluster != null) {
+          cluster.shutdown();
+        }
+      }
+    } else {
+      LOG.info("Not testing IPv6 binding as IPv6 us not supported");
+    }
   }
 
   @Test (timeout=300000)
@@ -111,7 +141,7 @@ public class TestNameNodeRespectsBindHostKeys {
     MiniDFSCluster cluster = null;
 
     LOG.info("Testing without " + DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY);
-    
+
     conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, LOCALHOST_SERVER_ADDRESS);
 
     // NN should not bind the wildcard address by default.
@@ -130,6 +160,29 @@ public class TestNameNodeRespectsBindHostKeys {
 
     LOG.info("Testing with " + DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY);
 
+    InetAddress localAddr = InetAddress.getLocalHost();
+    if (localAddr instanceof Inet6Address) {
+      // Tell NN to bind the IPv6 wildcard address.
+      conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY, IPv6_WILDCARD_ADDRESS);
+
+      // Verify that NN binds wildcard address now.
+      try {
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+        cluster.waitActive();
+        String address = getRpcServerAddress(cluster);
+        assertThat("Bind address " + address + " is not wildcard.",
+                   address, anyOf(
+                     is("/" + WILDCARD_ADDRESS),
+                     is("/" + IPv6_WILDCARD_ADDRESS)));
+      } finally {
+        if (cluster != null) {
+          cluster.shutdown();
+        }
+      }
+    } else {
+      LOG.info("Not testing IPv6 binding as IPv6 us not supported");
+    }
+
     // Tell NN to bind the wildcard address.
     conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY, WILDCARD_ADDRESS);
 
@@ -139,7 +192,9 @@ public class TestNameNodeRespectsBindHostKeys {
       cluster.waitActive();
       String address = getServiceRpcServerAddress(cluster);
       assertThat("Bind address " + address + " is not wildcard.",
-                 address, is("/" + WILDCARD_ADDRESS));
+                 address, anyOf(
+                   is("/" + WILDCARD_ADDRESS),
+                   is("/" + IPv6_WILDCARD_ADDRESS)));
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -159,7 +214,8 @@ public class TestNameNodeRespectsBindHostKeys {
       conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, LOCALHOST_SERVER_ADDRESS);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
-      String address = cluster.getNameNode().getHttpAddress().toString();
+      String address = NetUtils.getSocketAddressString(
+          cluster.getNameNode().getHttpAddress());
       assertFalse("HTTP Bind address not expected to be wildcard by default.",
                   address.startsWith(WILDCARD_ADDRESS));
     } finally {
@@ -179,7 +235,8 @@ public class TestNameNodeRespectsBindHostKeys {
       conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, LOCALHOST_SERVER_ADDRESS);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
-      String address = cluster.getNameNode().getHttpAddress().toString();
+      String address = NetUtils.getSocketAddressString(
+          cluster.getNameNode().getHttpAddress());
       assertTrue("HTTP Bind address " + address + " is not wildcard.",
                  address.startsWith(WILDCARD_ADDRESS));
     } finally {
@@ -233,7 +290,8 @@ public class TestNameNodeRespectsBindHostKeys {
       conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, LOCALHOST_SERVER_ADDRESS);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
-      String address = cluster.getNameNode().getHttpsAddress().toString();
+      String address = NetUtils.getSocketAddressString(
+          cluster.getNameNode().getHttpsAddress());
       assertFalse("HTTP Bind address not expected to be wildcard by default.",
                   address.startsWith(WILDCARD_ADDRESS));
     } finally {
@@ -253,7 +311,8 @@ public class TestNameNodeRespectsBindHostKeys {
       conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, LOCALHOST_SERVER_ADDRESS);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
-      String address = cluster.getNameNode().getHttpsAddress().toString();
+      String address = NetUtils
+          .getSocketAddressString(cluster.getNameNode().getHttpsAddress());
       assertTrue("HTTP Bind address " + address + " is not wildcard.",
                  address.startsWith(WILDCARD_ADDRESS));
     } finally {

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java

@@ -25,7 +25,9 @@
 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 static org.junit.Assert.assertThat;
+import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.AnyOf.anyOf;
 
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
@@ -43,14 +45,18 @@ public class TestNameNodeRpcServer {
     // 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.
+    // On IPv4-only machines it will return that it is listening on 0.0.0.0
+    // On dual-stack or IPv6-only machines it will return 0:0:0:0:0:0:0:0
     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());
+      String listenerAddress = ((NameNodeRpcServer)cluster.getNameNodeRpc())
+          .getClientRpcServer().getListenerAddress().getHostName();
+      assertThat("Bind address " + listenerAddress + " is not wildcard.",
+          listenerAddress, anyOf(is("0.0.0.0"), is("0:0:0:0:0:0:0:0")));
     } finally {
       if (cluster != null) {
         cluster.shutdown();