You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by br...@apache.org on 2021/07/26 12:17:10 UTC

[hadoop] branch HADOOP-17800 updated (1f1c38b -> 36b8ed1)

This is an automated email from the ASF dual-hosted git repository.

brahma pushed a change to branch HADOOP-17800
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


 discard 1f1c38b  HADOOP-11630. Allow hadoop.sh to bind to ipv6 conditionally. Contributed by Elliott Clark.
     add de41ce8  HDFS-16087. Fix stuck issue in rbfbalance tool (#3141).  Contributed by Eric Yin.
     add e634bf3  YARN-10630. [UI2] Ambiguous queue name resolution (#3214)
     add 0441efe  YARN-10860. Make max container per heartbeat configs refreshable. Contributed by Eric Badger.
     add dbd255f  HADOOP-17796. Upgrade jetty version to 9.4.43 (#3208)
     add 2da9b95  YARN-10657. We should make max application per queue to support node label. Contributed by Andras Gyori.
     add 98412ce  HADOOP-17813. Checkstyle - Allow line length: 100
     add 3a52bfc  HADOOP-17808. ipc.Client to set interrupt flag after catching InterruptedException (#3219)
     add aa1a5dd  YARN-10829. Support getApplications API in FederationClientInterceptor (#3135)
     add 63dfd84  HADOOP-17458. S3A to treat "SdkClientException: Data read has a different length than the expected" as EOFException (#3040)
     add 05b6a1a  YARN-10833. Set the X-FRAME-OPTIONS header for the default contexts. (#3203)
     add 4c35466  HADOOP-17317. [JDK 11] Upgrade dnsjava to remove illegal access warnings (#2442)
     add dd8e540  Addendum HADOOP-17770 WASB : Support disabling buffered reads in positional reads - Added the invalid SpotBugs warning to findbugs-exclude.xml (#3223)
     add 2f2f822  HDFS-12920. HDFS default value change (with adding time unit) breaks old version MR tarball work with new version (3.0) of hadoop. (#3227)
     add b7431c3  [UI2] Bump http-proxy to 1.18.1 (#2891)
     add 5d76549  HDFS-16131. Show storage type for failed volumes on namenode web (#3211). Contributed by  tomscut.
     add d710ec8  HDFS-16140. TestBootstrapAliasmap fails by BindException. (#3229)
     new 904c6ec  HADOOP-11630. Allow hadoop.sh to bind to ipv6 conditionally. Contributed by Elliott Clark.
     new 36b8ed1  HADOOP-12430. Fix HDFS client gets errors trying to to connect to IPv6 DataNode. Contributed by Nate Edel.

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (1f1c38b)
            \
             N -- N -- N   refs/heads/HADOOP-17800 (36b8ed1)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../src/main/resources/checkstyle/checkstyle.xml   |   4 +-
 .../main/java/org/apache/hadoop/ipc/Client.java    |   8 +-
 .../main/java/org/apache/hadoop/net/NetUtils.java  | 160 +++++++++++++++++++--
 .../org/apache/hadoop/security/SecurityUtil.java   |   7 +-
 .../java/org/apache/hadoop/net/TestNetUtils.java   |   8 +-
 .../hadoop/registry/server/dns/RegistryDNS.java    |  77 ++++------
 .../hadoop/registry/server/dns/SecureableZone.java |   3 +-
 .../registry/server/dns/TestRegistryDNS.java       | 118 +++++++--------
 .../apache/hadoop/hdfs/protocol/DatanodeID.java    |  14 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java    |   9 +-
 .../hdfs/rbfbalance/RouterDistCpProcedure.java     |   1 +
 .../hdfs/rbfbalance/TestRouterDistCpProcedure.java | 120 ++++++++++++++++
 .../hadoop/hdfs/server/datanode/DataNode.java      |   2 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java     |   4 +-
 .../src/main/resources/hdfs-default.xml            |  22 +--
 .../server/datanode/TestDataNodeVolumeFailure.java |   4 +
 .../TestDataNodeVolumeFailureReporting.java        |   5 +-
 .../server/namenode/ha/TestBootstrapAliasmap.java  |   3 +
 hadoop-project/pom.xml                             |   4 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java    |  16 ++-
 .../java/org/apache/hadoop/fs/s3a/TestInvoker.java |  36 +++++
 .../hadoop-azure/dev-support/findbugs-exclude.xml  |  13 ++
 .../hadoop/tools/fedbalance/DistCpProcedure.java   |   4 +-
 .../tools/fedbalance/TestDistCpProcedure.java      |   6 +-
 .../protocolrecords/GetApplicationsResponse.java   |  12 ++
 .../apache/hadoop/yarn/conf/YarnConfiguration.java |   9 ++
 .../yarn/conf/TestYarnConfigurationFields.java     |   2 +
 .../org/apache/hadoop/yarn/webapp/WebApps.java     |  31 ++--
 .../scheduler/capacity/AbstractCSQueue.java        |  46 +++---
 .../scheduler/capacity/CapacityScheduler.java      |  11 +-
 .../capacity/CapacitySchedulerQueueManager.java    |   5 +
 .../scheduler/capacity/LeafQueue.java              |   9 +-
 .../TestCapacitySchedulerAutoQueueCreation.java    |  10 +-
 .../scheduler/capacity/TestLeafQueue.java          |  39 +++++
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java    | 155 ++++++++------------
 .../clientrm/FederationClientInterceptor.java      |  48 ++++++-
 .../router/clientrm/RouterYarnClientUtils.java     | 137 ++++++++++++++++++
 .../clientrm/TestFederationClientInterceptor.java  | 118 ++++++++++++++-
 .../router/clientrm/TestRouterYarnClientUtils.java | 117 +++++++++++++++
 .../src/site/markdown/CapacityScheduler.md         |   4 +-
 .../main/webapp/app/components/tree-selector.js    |  14 +-
 .../webapp/app/models/yarn-queue/capacity-queue.js |   2 +-
 .../app/serializers/yarn-queue/capacity-queue.js   |  14 +-
 .../hadoop-yarn-ui/src/main/webapp/yarn.lock       |  22 ++-
 44 files changed, 1103 insertions(+), 350 deletions(-)
 create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/rbfbalance/TestRouterDistCpProcedure.java

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 02/02: HADOOP-12430. Fix HDFS client gets errors trying to to connect to IPv6 DataNode. Contributed by Nate Edel.

Posted by br...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

brahma pushed a commit to branch HADOOP-17800
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 36b8ed12a8502bdecc6b6ca16538d321969a5432
Author: Brahma Reddy Battula <br...@apache.org>
AuthorDate: Mon Jul 26 17:18:55 2021 +0530

    HADOOP-12430. Fix HDFS client gets errors trying to to connect to IPv6 DataNode. Contributed by Nate Edel.
---
 .../main/java/org/apache/hadoop/net/NetUtils.java  | 160 +++++++++++++++++++--
 .../java/org/apache/hadoop/net/TestNetUtils.java   |   8 +-
 .../apache/hadoop/hdfs/protocol/DatanodeID.java    |  14 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java    |   9 +-
 4 files changed, 162 insertions(+), 29 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
index 0f4dd9d..49fa540 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
@@ -40,7 +40,6 @@ import java.nio.channels.SocketChannel;
 import java.nio.channels.UnresolvedAddressException;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -61,6 +60,11 @@ import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.net.HostAndPort;
+import com.google.common.net.InetAddresses;
+import org.apache.http.conn.util.InetAddressUtils;
+import java.net.*;
+
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +74,7 @@ import org.slf4j.LoggerFactory;
 public class NetUtils {
   private static final Logger LOG = LoggerFactory.getLogger(NetUtils.class);
   
-  private static Map<String, String> hostToResolved = 
+  private static Map<String, String> hostToResolved =
                                      new HashMap<String, String>();
   /** text to point users elsewhere: {@value} */
   private static final String FOR_MORE_DETAILS_SEE
@@ -669,9 +673,6 @@ public class NetUtils {
     }
   }
 
-  private static final Pattern ipPortPattern = // Pattern for matching ip[:port]
-    Pattern.compile("\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}(:\\d+)?");
-  
   /**
    * Attempt to obtain the host name of the given string which contains
    * an IP address and an optional port.
@@ -680,16 +681,26 @@ public class NetUtils {
    * @return Host name or null if the name can not be determined
    */
   public static String getHostNameOfIP(String ipPort) {
-    if (null == ipPort || !ipPortPattern.matcher(ipPort).matches()) {
+    String ip = null;
+    if (null == ipPort || ipPort.isEmpty()) {
       return null;
     }
-    
     try {
-      int colonIdx = ipPort.indexOf(':');
-      String ip = (-1 == colonIdx) ? ipPort
-          : ipPort.substring(0, ipPort.indexOf(':'));
+      HostAndPort hostAndPort = HostAndPort.fromString(ipPort);
+      ip = hostAndPort.getHost();
+      if (!InetAddresses.isInetAddress(ip)) {
+        return null;
+      }
+    } catch (IllegalArgumentException e) {
+      LOG.debug("getHostNameOfIP: '" + ipPort
+              + "' is not a valid IP address or IP/Port pair.", e);
+      return null;
+    }
+
+    try {
       return InetAddress.getByName(ip).getHostName();
     } catch (UnknownHostException e) {
+      LOG.trace("getHostNameOfIP: '"+ipPort+"' name not resolved.", e);
       return null;
     }
   }
@@ -702,8 +713,20 @@ public class NetUtils {
    * @return host:port
    */
   public static String normalizeIP2HostName(String ipPort) {
-    if (null == ipPort || !ipPortPattern.matcher(ipPort).matches()) {
-      return ipPort;
+    String ip = null;
+    if (null == ipPort || ipPort.isEmpty()) {
+      return null;
+    }
+    try {
+      HostAndPort hostAndPort = HostAndPort.fromString(ipPort);
+      ip = hostAndPort.getHost();
+      if (!InetAddresses.isInetAddress(ip)) {
+        return null;
+      }
+    } catch (IllegalArgumentException e) {
+      LOG.debug("getHostNameOfIP: '" + ipPort
+          + "' is not a valid IP address or IP/Port pair.", e);
+      return null;
     }
 
     InetSocketAddress address = createSocketAddr(ipPort);
@@ -735,11 +758,88 @@ public class NetUtils {
 
   /**
    * Compose a "host:port" string from the address.
+   *
+   * Note that this preferentially returns the host name if available; if the
+   * IP address is desired, use getIPPortString(); if both are desired as in
+   * InetSocketAddress.toString, use getSocketAddressString()
    */
   public static String getHostPortString(InetSocketAddress addr) {
-    return addr.getHostName() + ":" + addr.getPort();
+    String hostName = addr.getHostName();
+    if (InetAddressUtils.isIPv6Address(hostName)) {
+      return "[" + hostName + "]:" + addr.getPort();
+    }
+    return hostName + ":" + addr.getPort();
   }
-  
+  /**
+   * Compose a "ip:port" string from the InetSocketAddress.
+   *
+   * Note that this may result in an NPE if passed an unresolved
+   * InetSocketAddress.
+   */
+  public static String getIPPortString(InetSocketAddress addr) {
+    final InetAddress ip = addr.getAddress();
+    // this is a judgement call, and we might arguably just guard against NPE
+    // by treating null as "" ; I think this is going to hide more bugs than it
+    // prevents
+    if (ip == null) {
+      throw new IllegalArgumentException(
+              "getIPPortString called with unresolved InetSocketAddress : "
+                      + getSocketAddressString(addr));
+    }
+    String ipString = ip.getHostAddress();
+    if (ip instanceof Inet6Address) {
+      return "[" + ipString + "]:" + addr.getPort();
+    }
+    return ipString + ":" + addr.getPort();
+  }
+
+  public static String getIPPortString(String ipAddr, int port) {
+    String s;
+    if (ipAddr != null) {
+      s = ipAddr + ":" + port;
+    } else {
+      s = ":" + port;
+    }
+    //Blank eventually will get to treated as localhost if this gets down to
+    // InetAddress. Tests extensively use a blank address, and we don't want
+    // to change behavior here.
+    if (ipAddr != null && !ipAddr.isEmpty() && InetAddressUtils
+        .isIPv6Address(ipAddr)) {
+      try {
+        InetAddress addr = InetAddress.getByName(ipAddr);
+        String cleanAddr = addr.getHostAddress();
+        if (addr instanceof Inet6Address) {
+          s = '[' + cleanAddr + ']' + ":" + port;
+        }
+      } catch (UnknownHostException e) {
+        // ignore anything that isn't an IPv6 literal and keep the old
+        // behavior. could add debug log here, but this should only happen
+        // if there's a bug in InetAddressUtils.isIPv6Address which accepts
+        // something that isn't an IPv6 literal.
+      }
+    }
+    return s;
+  }
+
+  /**
+   * An IPv6-safe version of InetSocketAddress.toString().
+   * Note that this will typically be of the form hostname/IP:port and is NOT
+   * a substitute for getHostPortString or getIPPortString.
+   */
+  public static String getSocketAddressString(InetSocketAddress addr) {
+    if (addr.isUnresolved()) {
+      return addr.toString();
+    }
+    InetAddress ip = addr.getAddress();
+    if (ip instanceof Inet6Address) {
+      String hostName = addr.getHostName();
+      return ((hostName != null) ? hostName : "")
+              + "/[" + ip.getHostAddress() + "]:" + addr.getPort();
+    } else {
+      return addr.toString();
+    }
+  }
+
   /**
    * Checks if {@code host} is a local host name and return {@link InetAddress}
    * corresponding to that address.
@@ -1037,6 +1137,38 @@ public class NetUtils {
   }
 
   /**
+   * Wrapper method on HostAndPort; returns the port from a host:port
+   * or IP:port pair.
+   *
+   * It's probably best to create your own HostAndPort.fromString(hp) and
+   * do a .getPort and .getHostText if you need both host and port in one
+   * scope.
+   */
+  public static int getPortFromHostPort(String hp) {
+    return HostAndPort.fromString(hp).getPort();
+  }
+
+  /**
+   * Wrapper method on HostAndPort; returns the host from a host:port
+   * or IP:port pair.
+   *
+   * It's probably best to create your own HostAndPort.fromString(hp) and
+   * do a .getPort and .getHostText if you need both host and port in one
+   * scope.
+   */
+  public static String getHostFromHostPort(String hp) {
+    return HostAndPort.fromString(hp).getHost();
+  }
+
+  public static InetAddress getInetAddressFromInetSocketAddressString(
+          String remoteAddr) {
+    int slashIdx = remoteAddr.indexOf('/') + 1;
+    int colonIdx = remoteAddr.lastIndexOf(':');
+    String ipOnly = remoteAddr.substring(slashIdx, colonIdx);
+    return InetAddresses.forString(ipOnly);
+  }
+
+  /**
    * Return an @{@link InetAddress} to bind to. If bindWildCardAddress is true
    * than returns null.
    *
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
index cfffd85..9840442 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
@@ -721,8 +721,8 @@ public class TestNetUtils {
     } catch (UnknownHostException e) {
       Assume.assumeTrue("Network not resolving "+ oneHost, false);
     }
-    List<String> hosts = Arrays.asList("127.0.0.1",
-        "localhost", oneHost, "UnknownHost123");
+    List<String> hosts = Arrays.asList(new String[] {"127.0.0.1",
+        "localhost", oneHost, "UnknownHost123.invalid"});
     List<String> normalizedHosts = NetUtils.normalizeHostNames(hosts);
     String summary = "original [" + StringUtils.join(hosts, ", ") + "]"
         + " normalized [" + StringUtils.join(normalizedHosts, ", ") + "]";
@@ -745,11 +745,13 @@ public class TestNetUtils {
     assertNull(NetUtils.getHostNameOfIP(null));
     assertNull(NetUtils.getHostNameOfIP(""));
     assertNull(NetUtils.getHostNameOfIP("crazytown"));
-    assertNull(NetUtils.getHostNameOfIP("127.0.0.1:"));   // no port
     assertNull(NetUtils.getHostNameOfIP("127.0.0.1:-1")); // bogus port
     assertNull(NetUtils.getHostNameOfIP("127.0.0.1:A"));  // bogus port
+    assertNotNull(NetUtils.getHostNameOfIP("[::1]"));
+    assertNotNull(NetUtils.getHostNameOfIP("[::1]:1"));
     assertNotNull(NetUtils.getHostNameOfIP("127.0.0.1"));
     assertNotNull(NetUtils.getHostNameOfIP("127.0.0.1:1"));
+    assertEquals("localhost", NetUtils.getHostNameOfIP("127.0.0.1:"));
   }
 
   @Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
index 2cb1687..ded7c9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-
+import org.apache.hadoop.net.NetUtils;
 import java.net.InetSocketAddress;
 
 /**
@@ -125,8 +125,9 @@ public class DatanodeID implements Comparable<DatanodeID> {
   }
 
   public void setIpAddr(String ipAddr) {
+    this.ipAddr = ipAddr;
     //updated during registration, preserve former xferPort
-    setIpAndXferPort(ipAddr, getByteString(ipAddr), xferPort);
+    setIpAndXferPort(this.ipAddr, getByteString(ipAddr), xferPort);
   }
 
   private void setIpAndXferPort(String ipAddr, ByteString ipAddrBytes,
@@ -135,7 +136,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
     this.ipAddr = ipAddr;
     this.ipAddrBytes = ipAddrBytes;
     this.xferPort = xferPort;
-    this.xferAddr = ipAddr + ":" + xferPort;
+    this.xferAddr = NetUtils.getIPPortString(ipAddr, xferPort);
   }
 
   public void setPeerHostName(String peerHostName) {
@@ -201,21 +202,21 @@ public class DatanodeID implements Comparable<DatanodeID> {
    * @return IP:ipcPort string
    */
   private String getIpcAddr() {
-    return ipAddr + ":" + ipcPort;
+    return NetUtils.getIPPortString(ipAddr, ipcPort);
   }
 
   /**
    * @return IP:infoPort string
    */
   public String getInfoAddr() {
-    return ipAddr + ":" + infoPort;
+    return NetUtils.getIPPortString(ipAddr, infoPort);
   }
 
   /**
    * @return IP:infoPort string
    */
   public String getInfoSecureAddr() {
-    return ipAddr + ":" + infoSecurePort;
+    return NetUtils.getIPPortString(ipAddr, infoSecurePort);
   }
 
   /**
@@ -299,6 +300,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
    * Note that this does not update storageID.
    */
   public void updateRegInfo(DatanodeID nodeReg) {
+    ipAddr = nodeReg.getIpAddr();
     setIpAndXferPort(nodeReg.getIpAddr(), nodeReg.getIpAddrBytes(),
         nodeReg.getXferPort());
     hostName = nodeReg.getHostName();
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
index 526f3d0..33f1bfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncr
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.HandshakeSecretProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherOptionProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslPropertiesResolver;
 import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection;
 import org.slf4j.Logger;
@@ -60,7 +61,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
-import org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses;
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
 
 /**
@@ -157,11 +157,8 @@ public final class DataTransferSaslUtil {
    * @return InetAddress from peer
    */
   public static InetAddress getPeerAddress(Peer peer) {
-    String remoteAddr = peer.getRemoteAddressString().split(":")[0];
-    int slashIdx = remoteAddr.indexOf('/');
-    return InetAddresses.forString(slashIdx != -1 ?
-        remoteAddr.substring(slashIdx + 1, remoteAddr.length()) :
-        remoteAddr);
+    String remoteAddr = peer.getRemoteAddressString();
+    return NetUtils.getInetAddressFromInetSocketAddressString(remoteAddr);
   }
 
   /**

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 01/02: HADOOP-11630. Allow hadoop.sh to bind to ipv6 conditionally. Contributed by Elliott Clark.

Posted by br...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

brahma pushed a commit to branch HADOOP-17800
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 904c6ec5044f4b8553a9ac4ec0dda056e0d9e795
Author: Brahma Reddy Battula <br...@apache.org>
AuthorDate: Tue Jul 20 19:39:42 2021 +0530

    HADOOP-11630. Allow hadoop.sh to bind to ipv6 conditionally. Contributed by Elliott Clark.
---
 .../hadoop-common/src/main/bin/hadoop-functions.sh                 | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index c4c3157..fd07f59 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -619,7 +619,12 @@ function hadoop_bootstrap
   export HADOOP_OS_TYPE=${HADOOP_OS_TYPE:-$(uname -s)}
 
   # defaults
-  export HADOOP_OPTS=${HADOOP_OPTS:-"-Djava.net.preferIPv4Stack=true"}
+  # shellcheck disable=SC2154
+  if [[ "${HADOOP_ALLOW_IPV6}" -ne "yes" ]]; then
+    export HADOOP_OPTS=${HADOOP_OPTS:-"-Djava.net.preferIPv4Stack=true"}
+  else
+    export HADOOP_OPTS=${HADOOP_OPTS:-""}
+  fi
   hadoop_debug "Initial HADOOP_OPTS=${HADOOP_OPTS}"
 }
 

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org