You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2022/06/14 09:46:46 UTC

[spark] branch master updated: [SPARK-39459][CORE] `local*HostName*` methods should support `IPv6`

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

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new f8c544b9ef7 [SPARK-39459][CORE] `local*HostName*` methods should support `IPv6`
f8c544b9ef7 is described below

commit f8c544b9ef78de37b1d149405ff6ead3285eac3e
Author: Dongjoon Hyun <do...@apache.org>
AuthorDate: Tue Jun 14 02:46:18 2022 -0700

    [SPARK-39459][CORE] `local*HostName*` methods should support `IPv6`
    
    ### What changes were proposed in this pull request?
    
    This PR aims to
    - Support `IPv6`-only environment in `localHostName`, `localHostNameForUri` and `localCanonicalHostName` methods
    - have no side-effects in `IPv4` environment.
    
    ### Why are the changes needed?
    
    Currently, Apache Spark fails on pure-IPv6 environment (which doesn't have IPv4 address).
    
    **BEFORE**
    ```
    $ SPARK_LOCAL_IP=::1 build/sbt "core/testOnly *.DistributedSuite" -Djava.net.preferIPv6Addresses=true
    ...
    Using SPARK_LOCAL_IP=::1
    ...
    [info] *** 45 TESTS FAILED ***
    [error] Failed tests:
    [error]         org.apache.spark.DistributedSuite
    [error] (core / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful
    [error] Total time: 9 s, completed Jun 13, 2022, 8:38:03 PM
    ```
    
    **AFTER**
    ```
    $ SPARK_LOCAL_IP=::1 build/sbt "core/testOnly *.DistributedSuite" -Djava.net.preferIPv6Addresses=true
    ...
    Using SPARK_LOCAL_IP=::1
    ...
    [info] Tests: succeeded 46, failed 0, canceled 0, ignored 0, pending 0
    [info] All tests passed.
    ```
    
    ### Does this PR introduce _any_ user-facing change?
    
    This will help IPv6-only environment users.
    
    ### How was this patch tested?
    
    Since we don't have IPv6 test CI, this should be tested in IPv6 environment manually with `DistributedSuite` and `Spark-Shell`.
    
    **DistributedSuite**
    ```
    $ SPARK_LOCAL_IP=::1 build/sbt "core/testOnly *.DistributedSuite" -Djava.net.preferIPv6Addresses=true
    ```
    
    **SPARK-SHELL**
    ```
    $ SPARK_LOCAL_IP=2600:...:...:c26a bin/spark-shell
    Setting default log level to "WARN".
    To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
    22/06/13 20:17:44 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
    Spark context Web UI available at http://unknown1498774f7f18.attlocal.net:4040
    Spark context available as 'sc' (master = local[*], app id = local-1655176664558).
    Spark session available as 'spark'.
    Welcome to
          ____              __
         / __/__  ___ _____/ /__
        _\ \/ _ \/ _ `/ __/  '_/
       /___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
          /_/
    
    Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 17.0.3)
    Type in expressions to have them evaluated.
    Type :help for more information.
    ```
    
    Closes #36863 from dongjoon-hyun/SPARK-39459.
    
    Authored-by: Dongjoon Hyun <do...@apache.org>
    Signed-off-by: Dongjoon Hyun <do...@apache.org>
---
 core/src/main/scala/org/apache/spark/util/Utils.scala | 15 ++++++++++++---
 1 file changed, 12 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index cf46a3d34c4..3e4a7e727a8 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1071,21 +1071,30 @@ private[spark] object Utils extends Logging {
    * Get the local machine's FQDN.
    */
   def localCanonicalHostName(): String = {
-    customHostname.getOrElse(localIpAddress.getCanonicalHostName)
+    addBracketsIfNeeded(customHostname.getOrElse(localIpAddress.getCanonicalHostName))
   }
 
   /**
    * Get the local machine's hostname.
+   * In case of IPv6, getHostAddress may return '0:0:0:0:0:0:0:1'.
    */
   def localHostName(): String = {
-    customHostname.getOrElse(localIpAddress.getHostAddress)
+    addBracketsIfNeeded(customHostname.getOrElse(localIpAddress.getHostAddress))
   }
 
   /**
    * Get the local machine's URI.
    */
   def localHostNameForURI(): String = {
-    customHostname.getOrElse(InetAddresses.toUriString(localIpAddress))
+    addBracketsIfNeeded(customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)))
+  }
+
+  private def addBracketsIfNeeded(addr: String): String = {
+    if (addr.contains(":") && !addr.contains("[")) {
+      "[" + addr + "]"
+    } else {
+      addr
+    }
   }
 
   /**


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