You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by va...@apache.org on 2018/05/09 17:48:14 UTC

[1/2] spark git commit: [SPARKR] Match pyspark features in SparkR communication protocol.

Repository: spark
Updated Branches:
  refs/heads/master 94155d039 -> 628c7b517


[SPARKR] Match pyspark features in SparkR communication protocol.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/628c7b51
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/628c7b51
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/628c7b51

Branch: refs/heads/master
Commit: 628c7b517969c4a7ccb26ea67ab3dd61266073ca
Parents: cc613b5
Author: Marcelo Vanzin <va...@cloudera.com>
Authored: Tue Apr 17 13:29:43 2018 -0700
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Wed May 9 10:47:35 2018 -0700

----------------------------------------------------------------------
 R/pkg/R/client.R                                |  4 +-
 R/pkg/R/deserialize.R                           | 10 ++--
 R/pkg/R/sparkR.R                                | 39 ++++++++++++--
 R/pkg/inst/worker/daemon.R                      |  4 +-
 R/pkg/inst/worker/worker.R                      |  5 +-
 .../org/apache/spark/api/r/RAuthHelper.scala    | 38 ++++++++++++++
 .../scala/org/apache/spark/api/r/RBackend.scala | 43 ++++++++++++---
 .../spark/api/r/RBackendAuthHandler.scala       | 55 ++++++++++++++++++++
 .../scala/org/apache/spark/api/r/RRunner.scala  | 35 +++++++++----
 .../scala/org/apache/spark/deploy/RRunner.scala |  6 ++-
 10 files changed, 210 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/R/pkg/R/client.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R
index 9d82814..7244cc9 100644
--- a/R/pkg/R/client.R
+++ b/R/pkg/R/client.R
@@ -19,7 +19,7 @@
 
 # Creates a SparkR client connection object
 # if one doesn't already exist
-connectBackend <- function(hostname, port, timeout) {
+connectBackend <- function(hostname, port, timeout, authSecret) {
   if (exists(".sparkRcon", envir = .sparkREnv)) {
     if (isOpen(.sparkREnv[[".sparkRCon"]])) {
       cat("SparkRBackend client connection already exists\n")
@@ -29,7 +29,7 @@ connectBackend <- function(hostname, port, timeout) {
 
   con <- socketConnection(host = hostname, port = port, server = FALSE,
                           blocking = TRUE, open = "wb", timeout = timeout)
-
+  doServerAuth(con, authSecret)
   assign(".sparkRCon", con, envir = .sparkREnv)
   con
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/R/pkg/R/deserialize.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R
index a90f7d3..cb03f16 100644
--- a/R/pkg/R/deserialize.R
+++ b/R/pkg/R/deserialize.R
@@ -60,14 +60,18 @@ readTypedObject <- function(con, type) {
     stop(paste("Unsupported type for deserialization", type)))
 }
 
-readString <- function(con) {
-  stringLen <- readInt(con)
-  raw <- readBin(con, raw(), stringLen, endian = "big")
+readStringData <- function(con, len) {
+  raw <- readBin(con, raw(), len, endian = "big")
   string <- rawToChar(raw)
   Encoding(string) <- "UTF-8"
   string
 }
 
+readString <- function(con) {
+  stringLen <- readInt(con)
+  readStringData(con, stringLen)
+}
+
 readInt <- function(con) {
   readBin(con, integer(), n = 1, endian = "big")
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/R/pkg/R/sparkR.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R
index a480ac6..38ee794 100644
--- a/R/pkg/R/sparkR.R
+++ b/R/pkg/R/sparkR.R
@@ -158,6 +158,10 @@ sparkR.sparkContext <- function(
                     " please use the --packages commandline instead", sep = ","))
     }
     backendPort <- existingPort
+    authSecret <- Sys.getenv("SPARKR_BACKEND_AUTH_SECRET")
+    if (nchar(authSecret) == 0) {
+      stop("Auth secret not provided in environment.")
+    }
   } else {
     path <- tempfile(pattern = "backend_port")
     submitOps <- getClientModeSparkSubmitOpts(
@@ -186,16 +190,27 @@ sparkR.sparkContext <- function(
     monitorPort <- readInt(f)
     rLibPath <- readString(f)
     connectionTimeout <- readInt(f)
+
+    # Don't use readString() so that we can provide a useful
+    # error message if the R and Java versions are mismatched.
+    authSecretLen = readInt(f)
+    if (length(authSecretLen) == 0 || authSecretLen == 0) {
+      stop("Unexpected EOF in JVM connection data. Mismatched versions?")
+    }
+    authSecret <- readStringData(f, authSecretLen)
     close(f)
     file.remove(path)
     if (length(backendPort) == 0 || backendPort == 0 ||
         length(monitorPort) == 0 || monitorPort == 0 ||
-        length(rLibPath) != 1) {
+        length(rLibPath) != 1 || length(authSecret) == 0) {
       stop("JVM failed to launch")
     }
-    assign(".monitorConn",
-           socketConnection(port = monitorPort, timeout = connectionTimeout),
-           envir = .sparkREnv)
+
+    monitorConn <- socketConnection(port = monitorPort, blocking = TRUE,
+                                    timeout = connectionTimeout, open = "wb")
+    doServerAuth(monitorConn, authSecret)
+
+    assign(".monitorConn", monitorConn, envir = .sparkREnv)
     assign(".backendLaunched", 1, envir = .sparkREnv)
     if (rLibPath != "") {
       assign(".libPath", rLibPath, envir = .sparkREnv)
@@ -205,7 +220,7 @@ sparkR.sparkContext <- function(
 
   .sparkREnv$backendPort <- backendPort
   tryCatch({
-    connectBackend("localhost", backendPort, timeout = connectionTimeout)
+    connectBackend("localhost", backendPort, timeout = connectionTimeout, authSecret = authSecret)
   },
   error = function(err) {
     stop("Failed to connect JVM\n")
@@ -687,3 +702,17 @@ sparkCheckInstall <- function(sparkHome, master, deployMode) {
     NULL
   }
 }
+
+# Utility function for sending auth data over a socket and checking the server's reply.
+doServerAuth <- function(con, authSecret) {
+  if (nchar(authSecret) == 0) {
+    stop("Auth secret not provided.")
+  }
+  writeString(con, authSecret)
+  flush(con)
+  reply <- readString(con)
+  if (reply != "ok") {
+    close(con)
+    stop("Unexpected reply from server.")
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/R/pkg/inst/worker/daemon.R
----------------------------------------------------------------------
diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R
index 2e31dc5..fb9db63 100644
--- a/R/pkg/inst/worker/daemon.R
+++ b/R/pkg/inst/worker/daemon.R
@@ -28,7 +28,9 @@ suppressPackageStartupMessages(library(SparkR))
 
 port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT"))
 inputCon <- socketConnection(
-    port = port, open = "rb", blocking = TRUE, timeout = connectionTimeout)
+    port = port, open = "wb", blocking = TRUE, timeout = connectionTimeout)
+
+SparkR:::doServerAuth(inputCon, Sys.getenv("SPARKR_WORKER_SECRET"))
 
 # Waits indefinitely for a socket connecion by default.
 selectTimeout <- NULL

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/R/pkg/inst/worker/worker.R
----------------------------------------------------------------------
diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R
index 00789d8..ba458d2 100644
--- a/R/pkg/inst/worker/worker.R
+++ b/R/pkg/inst/worker/worker.R
@@ -100,9 +100,12 @@ suppressPackageStartupMessages(library(SparkR))
 
 port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT"))
 inputCon <- socketConnection(
-    port = port, blocking = TRUE, open = "rb", timeout = connectionTimeout)
+    port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout)
+SparkR:::doServerAuth(inputCon, Sys.getenv("SPARKR_WORKER_SECRET"))
+
 outputCon <- socketConnection(
     port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout)
+SparkR:::doServerAuth(outputCon, Sys.getenv("SPARKR_WORKER_SECRET"))
 
 # read the index of the current partition inside the RDD
 partition <- SparkR:::readInt(inputCon)

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/core/src/main/scala/org/apache/spark/api/r/RAuthHelper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/r/RAuthHelper.scala b/core/src/main/scala/org/apache/spark/api/r/RAuthHelper.scala
new file mode 100644
index 0000000..ac6826a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/r/RAuthHelper.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api.r
+
+import java.io.{DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import org.apache.spark.SparkConf
+import org.apache.spark.security.SocketAuthHelper
+
+private[spark] class RAuthHelper(conf: SparkConf) extends SocketAuthHelper(conf) {
+
+  override protected def readUtf8(s: Socket): String = {
+    SerDe.readString(new DataInputStream(s.getInputStream()))
+  }
+
+  override protected def writeUtf8(str: String, s: Socket): Unit = {
+    val out = s.getOutputStream()
+    SerDe.writeString(new DataOutputStream(out), str)
+    out.flush()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
index 2d1152a..3b2e809 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
@@ -17,8 +17,8 @@
 
 package org.apache.spark.api.r
 
-import java.io.{DataOutputStream, File, FileOutputStream, IOException}
-import java.net.{InetAddress, InetSocketAddress, ServerSocket}
+import java.io.{DataInputStream, DataOutputStream, File, FileOutputStream, IOException}
+import java.net.{InetAddress, InetSocketAddress, ServerSocket, Socket}
 import java.util.concurrent.TimeUnit
 
 import io.netty.bootstrap.ServerBootstrap
@@ -32,6 +32,8 @@ import io.netty.handler.timeout.ReadTimeoutHandler
 
 import org.apache.spark.SparkConf
 import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.util.Utils
 
 /**
  * Netty-based backend server that is used to communicate between R and Java.
@@ -45,7 +47,7 @@ private[spark] class RBackend {
   /** Tracks JVM objects returned to R for this RBackend instance. */
   private[r] val jvmObjectTracker = new JVMObjectTracker
 
-  def init(): Int = {
+  def init(): (Int, RAuthHelper) = {
     val conf = new SparkConf()
     val backendConnectionTimeout = conf.getInt(
       "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT)
@@ -53,6 +55,7 @@ private[spark] class RBackend {
       conf.getInt("spark.r.numRBackendThreads", SparkRDefaults.DEFAULT_NUM_RBACKEND_THREADS))
     val workerGroup = bossGroup
     val handler = new RBackendHandler(this)
+    val authHelper = new RAuthHelper(conf)
 
     bootstrap = new ServerBootstrap()
       .group(bossGroup, workerGroup)
@@ -71,13 +74,16 @@ private[spark] class RBackend {
             new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
           .addLast("decoder", new ByteArrayDecoder())
           .addLast("readTimeoutHandler", new ReadTimeoutHandler(backendConnectionTimeout))
+          .addLast(new RBackendAuthHandler(authHelper.secret))
           .addLast("handler", handler)
       }
     })
 
     channelFuture = bootstrap.bind(new InetSocketAddress("localhost", 0))
     channelFuture.syncUninterruptibly()
-    channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort()
+
+    val port = channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort()
+    (port, authHelper)
   }
 
   def run(): Unit = {
@@ -116,7 +122,7 @@ private[spark] object RBackend extends Logging {
     val sparkRBackend = new RBackend()
     try {
       // bind to random port
-      val boundPort = sparkRBackend.init()
+      val (boundPort, authHelper) = sparkRBackend.init()
       val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost"))
       val listenPort = serverSocket.getLocalPort()
       // Connection timeout is set by socket client. To make it configurable we will pass the
@@ -133,6 +139,7 @@ private[spark] object RBackend extends Logging {
       dos.writeInt(listenPort)
       SerDe.writeString(dos, RUtils.rPackages.getOrElse(""))
       dos.writeInt(backendConnectionTimeout)
+      SerDe.writeString(dos, authHelper.secret)
       dos.close()
       f.renameTo(new File(path))
 
@@ -144,12 +151,35 @@ private[spark] object RBackend extends Logging {
           val buf = new Array[Byte](1024)
           // shutdown JVM if R does not connect back in 10 seconds
           serverSocket.setSoTimeout(10000)
+
+          // Wait for the R process to connect back, ignoring any failed auth attempts. Allow
+          // a max number of connection attempts to avoid looping forever.
           try {
-            val inSocket = serverSocket.accept()
+            var remainingAttempts = 10
+            var inSocket: Socket = null
+            while (inSocket == null) {
+              inSocket = serverSocket.accept()
+              try {
+                authHelper.authClient(inSocket)
+              } catch {
+                case e: Exception =>
+                  remainingAttempts -= 1
+                  if (remainingAttempts == 0) {
+                    val msg = "Too many failed authentication attempts."
+                    logError(msg)
+                    throw new IllegalStateException(msg)
+                  }
+                  logInfo("Client connection failed authentication.")
+                  inSocket = null
+              }
+            }
+
             serverSocket.close()
+
             // wait for the end of socket, closed if R process die
             inSocket.getInputStream().read(buf)
           } finally {
+            serverSocket.close()
             sparkRBackend.close()
             System.exit(0)
           }
@@ -165,4 +195,5 @@ private[spark] object RBackend extends Logging {
     }
     System.exit(0)
   }
+
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala
new file mode 100644
index 0000000..4162e4a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api.r
+
+import java.io.{ByteArrayOutputStream, DataOutputStream}
+import java.nio.charset.StandardCharsets.UTF_8
+
+import io.netty.channel.{Channel, ChannelHandlerContext, SimpleChannelInboundHandler}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * Authentication handler for connections from the R process.
+ */
+private class RBackendAuthHandler(secret: String)
+  extends SimpleChannelInboundHandler[Array[Byte]] with Logging {
+
+  override def channelRead0(ctx: ChannelHandlerContext, msg: Array[Byte]): Unit = {
+    // The R code adds a null terminator to serialized strings, so ignore it here.
+    val clientSecret = new String(msg, 0, msg.length - 1, UTF_8)
+    try {
+      require(secret == clientSecret, "Auth secret mismatch.")
+      ctx.pipeline().remove(this)
+      writeReply("ok", ctx.channel())
+    } catch {
+      case e: Exception =>
+        logInfo("Authentication failure.", e)
+        writeReply("err", ctx.channel())
+        ctx.close()
+    }
+  }
+
+  private def writeReply(reply: String, chan: Channel): Unit = {
+    val out = new ByteArrayOutputStream()
+    SerDe.writeString(new DataOutputStream(out), reply)
+    chan.writeAndFlush(out.toByteArray())
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
index 8811839..e7fdc39 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
@@ -74,14 +74,19 @@ private[spark] class RRunner[U](
 
     // the socket used to send out the input of task
     serverSocket.setSoTimeout(10000)
-    val inSocket = serverSocket.accept()
-    startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex)
-
-    // the socket used to receive the output of task
-    val outSocket = serverSocket.accept()
-    val inputStream = new BufferedInputStream(outSocket.getInputStream)
-    dataStream = new DataInputStream(inputStream)
-    serverSocket.close()
+    dataStream = try {
+      val inSocket = serverSocket.accept()
+      RRunner.authHelper.authClient(inSocket)
+      startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex)
+
+      // the socket used to receive the output of task
+      val outSocket = serverSocket.accept()
+      RRunner.authHelper.authClient(outSocket)
+      val inputStream = new BufferedInputStream(outSocket.getInputStream)
+      new DataInputStream(inputStream)
+    } finally {
+      serverSocket.close()
+    }
 
     try {
       return new Iterator[U] {
@@ -315,6 +320,11 @@ private[r] object RRunner {
   private[this] var errThread: BufferedStreamThread = _
   private[this] var daemonChannel: DataOutputStream = _
 
+  private lazy val authHelper = {
+    val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf())
+    new RAuthHelper(conf)
+  }
+
   /**
    * Start a thread to print the process's stderr to ours
    */
@@ -349,6 +359,7 @@ private[r] object RRunner {
     pb.environment().put("SPARKR_BACKEND_CONNECTION_TIMEOUT", rConnectionTimeout.toString)
     pb.environment().put("SPARKR_SPARKFILES_ROOT_DIR", SparkFiles.getRootDirectory())
     pb.environment().put("SPARKR_IS_RUNNING_ON_WORKER", "TRUE")
+    pb.environment().put("SPARKR_WORKER_SECRET", authHelper.secret)
     pb.redirectErrorStream(true)  // redirect stderr into stdout
     val proc = pb.start()
     val errThread = startStdoutThread(proc)
@@ -370,8 +381,12 @@ private[r] object RRunner {
           // the socket used to send out the input of task
           serverSocket.setSoTimeout(10000)
           val sock = serverSocket.accept()
-          daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
-          serverSocket.close()
+          try {
+            authHelper.authClient(sock)
+            daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
+          } finally {
+            serverSocket.close()
+          }
         }
         try {
           daemonChannel.writeInt(port)

http://git-wip-us.apache.org/repos/asf/spark/blob/628c7b51/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
index 6eb53a8..e86b362 100644
--- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
@@ -68,10 +68,13 @@ object RRunner {
     // Java system properties etc.
     val sparkRBackend = new RBackend()
     @volatile var sparkRBackendPort = 0
+    @volatile var sparkRBackendSecret: String = null
     val initialized = new Semaphore(0)
     val sparkRBackendThread = new Thread("SparkR backend") {
       override def run() {
-        sparkRBackendPort = sparkRBackend.init()
+        val (port, authHelper) = sparkRBackend.init()
+        sparkRBackendPort = port
+        sparkRBackendSecret = authHelper.secret
         initialized.release()
         sparkRBackend.run()
       }
@@ -91,6 +94,7 @@ object RRunner {
         env.put("SPARKR_PACKAGE_DIR", rPackageDir.mkString(","))
         env.put("R_PROFILE_USER",
           Seq(rPackageDir(0), "SparkR", "profile", "general.R").mkString(File.separator))
+        env.put("SPARKR_BACKEND_AUTH_SECRET", sparkRBackendSecret)
         builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize
         val process = builder.start()
 


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


[2/2] spark git commit: [PYSPARK] Update py4j to version 0.10.7.

Posted by va...@apache.org.
[PYSPARK] Update py4j to version 0.10.7.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cc613b55
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cc613b55
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cc613b55

Branch: refs/heads/master
Commit: cc613b552e753d03cb62661591de59e1c8d82c74
Parents: 94155d0
Author: Marcelo Vanzin <va...@cloudera.com>
Authored: Fri Apr 13 14:28:24 2018 -0700
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Wed May 9 10:47:35 2018 -0700

----------------------------------------------------------------------
 LICENSE                                         |   2 +-
 bin/pyspark                                     |   6 +-
 bin/pyspark2.cmd                                |   2 +-
 core/pom.xml                                    |   2 +-
 .../org/apache/spark/SecurityManager.scala      |  12 +--
 .../spark/api/python/PythonGatewayServer.scala  |  50 ++++++---
 .../org/apache/spark/api/python/PythonRDD.scala |  29 ++++--
 .../apache/spark/api/python/PythonUtils.scala   |   2 +-
 .../spark/api/python/PythonWorkerFactory.scala  |  20 ++--
 .../org/apache/spark/deploy/PythonRunner.scala  |  12 ++-
 .../apache/spark/internal/config/package.scala  |   5 +
 .../spark/security/SocketAuthHelper.scala       | 101 +++++++++++++++++++
 .../scala/org/apache/spark/util/Utils.scala     |  13 ++-
 .../spark/security/SocketAuthHelperSuite.scala  |  97 ++++++++++++++++++
 dev/deps/spark-deps-hadoop-2.6                  |   2 +-
 dev/deps/spark-deps-hadoop-2.7                  |   2 +-
 dev/deps/spark-deps-hadoop-3.1                  |   2 +-
 dev/run-pip-tests                               |   2 +-
 python/README.md                                |   2 +-
 python/docs/Makefile                            |   2 +-
 python/lib/py4j-0.10.6-src.zip                  | Bin 80352 -> 0 bytes
 python/lib/py4j-0.10.7-src.zip                  | Bin 0 -> 42437 bytes
 python/pyspark/context.py                       |   4 +-
 python/pyspark/daemon.py                        |  21 +++-
 python/pyspark/java_gateway.py                  |  93 ++++++++++-------
 python/pyspark/rdd.py                           |  21 ++--
 python/pyspark/sql/dataframe.py                 |  12 +--
 python/pyspark/worker.py                        |   7 +-
 python/setup.py                                 |   2 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |   2 +-
 .../spark/deploy/yarn/YarnClusterSuite.scala    |   2 +-
 sbin/spark-config.sh                            |   2 +-
 .../scala/org/apache/spark/sql/Dataset.scala    |   6 +-
 33 files changed, 417 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index c2b0d72..820f14d 100644
--- a/LICENSE
+++ b/LICENSE
@@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
      (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf)
      (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net)
      (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net)
-     (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.6 - http://py4j.sourceforge.net/)
+     (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.7 - http://py4j.sourceforge.net/)
      (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/)
      (BSD licence) sbt and sbt-launch-lib.bash
      (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE)

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/bin/pyspark
----------------------------------------------------------------------
diff --git a/bin/pyspark b/bin/pyspark
index dd28627..5d5affb 100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@ -25,14 +25,14 @@ source "${SPARK_HOME}"/bin/load-spark-env.sh
 export _SPARK_CMD_USAGE="Usage: ./bin/pyspark [options]"
 
 # In Spark 2.0, IPYTHON and IPYTHON_OPTS are removed and pyspark fails to launch if either option
-# is set in the user's environment. Instead, users should set PYSPARK_DRIVER_PYTHON=ipython 
+# is set in the user's environment. Instead, users should set PYSPARK_DRIVER_PYTHON=ipython
 # to use IPython and set PYSPARK_DRIVER_PYTHON_OPTS to pass options when starting the Python driver
 # (e.g. PYSPARK_DRIVER_PYTHON_OPTS='notebook').  This supports full customization of the IPython
 # and executor Python executables.
 
 # Fail noisily if removed options are set
 if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then
-  echo "Error in pyspark startup:" 
+  echo "Error in pyspark startup:"
   echo "IPYTHON and IPYTHON_OPTS are removed in Spark 2.0+. Remove these from the environment and set PYSPARK_DRIVER_PYTHON and PYSPARK_DRIVER_PYTHON_OPTS instead."
   exit 1
 fi
@@ -57,7 +57,7 @@ export PYSPARK_PYTHON
 
 # Add the PySpark classes to the Python path:
 export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH"
-export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:$PYTHONPATH"
+export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-src.zip:$PYTHONPATH"
 
 # Load the PySpark shell.py script when ./pyspark is used interactively:
 export OLD_PYTHONSTARTUP="$PYTHONSTARTUP"

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/bin/pyspark2.cmd
----------------------------------------------------------------------
diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd
index 663670f..15fa910 100644
--- a/bin/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" (
 )
 
 set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH%
-set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.6-src.zip;%PYTHONPATH%
+set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.7-src.zip;%PYTHONPATH%
 
 set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
 set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 093a986..220522d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -350,7 +350,7 @@
     <dependency>
       <groupId>net.sf.py4j</groupId>
       <artifactId>py4j</artifactId>
-      <version>0.10.6</version>
+      <version>0.10.7</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/SecurityManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index dbfd5a5..b874763 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -17,15 +17,10 @@
 
 package org.apache.spark
 
-import java.lang.{Byte => JByte}
 import java.net.{Authenticator, PasswordAuthentication}
 import java.nio.charset.StandardCharsets.UTF_8
-import java.security.{KeyStore, SecureRandom}
-import java.security.cert.X509Certificate
 import javax.net.ssl._
 
-import com.google.common.hash.HashCodes
-import com.google.common.io.Files
 import org.apache.hadoop.io.Text
 import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 
@@ -365,13 +360,8 @@ private[spark] class SecurityManager(
         return
     }
 
-    val rnd = new SecureRandom()
-    val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE
-    val secretBytes = new Array[Byte](length)
-    rnd.nextBytes(secretBytes)
-
+    secretKey = Utils.createSecret(sparkConf)
     val creds = new Credentials()
-    secretKey = HashCodes.fromBytes(secretBytes).toString()
     creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8))
     UserGroupInformation.getCurrentUser().addCredentials(creds)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala
index 11f2432..9ddc4a4 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala
@@ -17,26 +17,39 @@
 
 package org.apache.spark.api.python
 
-import java.io.DataOutputStream
-import java.net.Socket
+import java.io.{DataOutputStream, File, FileOutputStream}
+import java.net.InetAddress
+import java.nio.charset.StandardCharsets.UTF_8
+import java.nio.file.Files
 
 import py4j.GatewayServer
 
+import org.apache.spark.SparkConf
 import org.apache.spark.internal.Logging
 import org.apache.spark.util.Utils
 
 /**
- * Process that starts a Py4J GatewayServer on an ephemeral port and communicates the bound port
- * back to its caller via a callback port specified by the caller.
+ * Process that starts a Py4J GatewayServer on an ephemeral port.
  *
  * This process is launched (via SparkSubmit) by the PySpark driver (see java_gateway.py).
  */
 private[spark] object PythonGatewayServer extends Logging {
   initializeLogIfNecessary(true)
 
-  def main(args: Array[String]): Unit = Utils.tryOrExit {
-    // Start a GatewayServer on an ephemeral port
-    val gatewayServer: GatewayServer = new GatewayServer(null, 0)
+  def main(args: Array[String]): Unit = {
+    val secret = Utils.createSecret(new SparkConf())
+
+    // Start a GatewayServer on an ephemeral port. Make sure the callback client is configured
+    // with the same secret, in case the app needs callbacks from the JVM to the underlying
+    // python processes.
+    val localhost = InetAddress.getLoopbackAddress()
+    val gatewayServer: GatewayServer = new GatewayServer.GatewayServerBuilder()
+      .authToken(secret)
+      .javaPort(0)
+      .javaAddress(localhost)
+      .callbackClient(GatewayServer.DEFAULT_PYTHON_PORT, localhost, secret)
+      .build()
+
     gatewayServer.start()
     val boundPort: Int = gatewayServer.getListeningPort
     if (boundPort == -1) {
@@ -46,15 +59,24 @@ private[spark] object PythonGatewayServer extends Logging {
       logDebug(s"Started PythonGatewayServer on port $boundPort")
     }
 
-    // Communicate the bound port back to the caller via the caller-specified callback port
-    val callbackHost = sys.env("_PYSPARK_DRIVER_CALLBACK_HOST")
-    val callbackPort = sys.env("_PYSPARK_DRIVER_CALLBACK_PORT").toInt
-    logDebug(s"Communicating GatewayServer port to Python driver at $callbackHost:$callbackPort")
-    val callbackSocket = new Socket(callbackHost, callbackPort)
-    val dos = new DataOutputStream(callbackSocket.getOutputStream)
+    // Communicate the connection information back to the python process by writing the
+    // information in the requested file. This needs to match the read side in java_gateway.py.
+    val connectionInfoPath = new File(sys.env("_PYSPARK_DRIVER_CONN_INFO_PATH"))
+    val tmpPath = Files.createTempFile(connectionInfoPath.getParentFile().toPath(),
+      "connection", ".info").toFile()
+
+    val dos = new DataOutputStream(new FileOutputStream(tmpPath))
     dos.writeInt(boundPort)
+
+    val secretBytes = secret.getBytes(UTF_8)
+    dos.writeInt(secretBytes.length)
+    dos.write(secretBytes, 0, secretBytes.length)
     dos.close()
-    callbackSocket.close()
+
+    if (!tmpPath.renameTo(connectionInfoPath)) {
+      logError(s"Unable to write connection information to $connectionInfoPath.")
+      System.exit(1)
+    }
 
     // Exit on EOF or broken pipe to ensure that this process dies when the Python driver dies:
     while (System.in.read() != -1) {

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index f6293c0..a1ee2f7 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -38,6 +38,7 @@ import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.input.PortableDataStream
 import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.RDD
+import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util._
 
 
@@ -107,6 +108,12 @@ private[spark] object PythonRDD extends Logging {
   // remember the broadcasts sent to each worker
   private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]()
 
+  // Authentication helper used when serving iterator data.
+  private lazy val authHelper = {
+    val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf())
+    new SocketAuthHelper(conf)
+  }
+
   def getWorkerBroadcasts(worker: Socket): mutable.Set[Long] = {
     synchronized {
       workerBroadcasts.getOrElseUpdate(worker, new mutable.HashSet[Long]())
@@ -129,12 +136,13 @@ private[spark] object PythonRDD extends Logging {
    * (effectively a collect()), but allows you to run on a certain subset of partitions,
    * or to enable local execution.
    *
-   * @return the port number of a local socket which serves the data collected from this job.
+   * @return 2-tuple (as a Java array) with the port number of a local socket which serves the
+   *         data collected from this job, and the secret for authentication.
    */
   def runJob(
       sc: SparkContext,
       rdd: JavaRDD[Array[Byte]],
-      partitions: JArrayList[Int]): Int = {
+      partitions: JArrayList[Int]): Array[Any] = {
     type ByteArray = Array[Byte]
     type UnrolledPartition = Array[ByteArray]
     val allPartitions: Array[UnrolledPartition] =
@@ -147,13 +155,14 @@ private[spark] object PythonRDD extends Logging {
   /**
    * A helper function to collect an RDD as an iterator, then serve it via socket.
    *
-   * @return the port number of a local socket which serves the data collected from this job.
+   * @return 2-tuple (as a Java array) with the port number of a local socket which serves the
+   *         data collected from this job, and the secret for authentication.
    */
-  def collectAndServe[T](rdd: RDD[T]): Int = {
+  def collectAndServe[T](rdd: RDD[T]): Array[Any] = {
     serveIterator(rdd.collect().iterator, s"serve RDD ${rdd.id}")
   }
 
-  def toLocalIteratorAndServe[T](rdd: RDD[T]): Int = {
+  def toLocalIteratorAndServe[T](rdd: RDD[T]): Array[Any] = {
     serveIterator(rdd.toLocalIterator, s"serve toLocalIterator")
   }
 
@@ -384,8 +393,11 @@ private[spark] object PythonRDD extends Logging {
    * and send them into this connection.
    *
    * The thread will terminate after all the data are sent or any exceptions happen.
+   *
+   * @return 2-tuple (as a Java array) with the port number of a local socket which serves the
+   *         data collected from this job, and the secret for authentication.
    */
-  def serveIterator[T](items: Iterator[T], threadName: String): Int = {
+  def serveIterator(items: Iterator[_], threadName: String): Array[Any] = {
     val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost"))
     // Close the socket if no connection in 15 seconds
     serverSocket.setSoTimeout(15000)
@@ -395,11 +407,14 @@ private[spark] object PythonRDD extends Logging {
       override def run() {
         try {
           val sock = serverSocket.accept()
+          authHelper.authClient(sock)
+
           val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
           Utils.tryWithSafeFinally {
             writeIteratorToStream(items, out)
           } {
             out.close()
+            sock.close()
           }
         } catch {
           case NonFatal(e) =>
@@ -410,7 +425,7 @@ private[spark] object PythonRDD extends Logging {
       }
     }.start()
 
-    serverSocket.getLocalPort
+    Array(serverSocket.getLocalPort, authHelper.secret)
   }
 
   private def getMergedConf(confAsMap: java.util.HashMap[String, String],

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
index 92e228a..27a5e19 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala
@@ -32,7 +32,7 @@ private[spark] object PythonUtils {
     val pythonPath = new ArrayBuffer[String]
     for (sparkHome <- sys.env.get("SPARK_HOME")) {
       pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator)
-      pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.6-src.zip").mkString(File.separator)
+      pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.7-src.zip").mkString(File.separator)
     }
     pythonPath ++= SparkContext.jarOfObject(this)
     pythonPath.mkString(File.pathSeparator)

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 2340580..6afa37a 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable
 
 import org.apache.spark._
 import org.apache.spark.internal.Logging
+import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util.{RedirectThread, Utils}
 
 private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String])
@@ -67,6 +68,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
     value
   }.getOrElse("pyspark.worker")
 
+  private val authHelper = new SocketAuthHelper(SparkEnv.get.conf)
+
   var daemon: Process = null
   val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1))
   var daemonPort: Int = 0
@@ -108,6 +111,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
       if (pid < 0) {
         throw new IllegalStateException("Python daemon failed to launch worker with code " + pid)
       }
+
+      authHelper.authToServer(socket)
       daemonWorkers.put(socket, pid)
       socket
     }
@@ -145,25 +150,24 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
       workerEnv.put("PYTHONPATH", pythonPath)
       // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u:
       workerEnv.put("PYTHONUNBUFFERED", "YES")
+      workerEnv.put("PYTHON_WORKER_FACTORY_PORT", serverSocket.getLocalPort.toString)
+      workerEnv.put("PYTHON_WORKER_FACTORY_SECRET", authHelper.secret)
       val worker = pb.start()
 
       // Redirect worker stdout and stderr
       redirectStreamsToStderr(worker.getInputStream, worker.getErrorStream)
 
-      // Tell the worker our port
-      val out = new  OutputStreamWriter(worker.getOutputStream, StandardCharsets.UTF_8)
-      out.write(serverSocket.getLocalPort + "\n")
-      out.flush()
-
-      // Wait for it to connect to our socket
+      // Wait for it to connect to our socket, and validate the auth secret.
       serverSocket.setSoTimeout(10000)
+
       try {
         val socket = serverSocket.accept()
+        authHelper.authClient(socket)
         simpleWorkers.put(socket, worker)
         return socket
       } catch {
         case e: Exception =>
-          throw new SparkException("Python worker did not connect back in time", e)
+          throw new SparkException("Python worker failed to connect back.", e)
       }
     } finally {
       if (serverSocket != null) {
@@ -187,6 +191,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
         val workerEnv = pb.environment()
         workerEnv.putAll(envVars.asJava)
         workerEnv.put("PYTHONPATH", pythonPath)
+        workerEnv.put("PYTHON_WORKER_FACTORY_SECRET", authHelper.secret)
         // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u:
         workerEnv.put("PYTHONUNBUFFERED", "YES")
         daemon = pb.start()
@@ -218,7 +223,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
 
         // Redirect daemon stdout and stderr
         redirectStreamsToStderr(in, daemon.getErrorStream)
-
       } catch {
         case e: Exception =>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala
index 7aca305..1b7e031 100644
--- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.deploy
 
 import java.io.File
-import java.net.URI
+import java.net.{InetAddress, URI}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
@@ -39,6 +39,7 @@ object PythonRunner {
     val pyFiles = args(1)
     val otherArgs = args.slice(2, args.length)
     val sparkConf = new SparkConf()
+    val secret = Utils.createSecret(sparkConf)
     val pythonExec = sparkConf.get(PYSPARK_DRIVER_PYTHON)
       .orElse(sparkConf.get(PYSPARK_PYTHON))
       .orElse(sys.env.get("PYSPARK_DRIVER_PYTHON"))
@@ -51,7 +52,13 @@ object PythonRunner {
 
     // Launch a Py4J gateway server for the process to connect to; this will let it see our
     // Java system properties and such
-    val gatewayServer = new py4j.GatewayServer(null, 0)
+    val localhost = InetAddress.getLoopbackAddress()
+    val gatewayServer = new py4j.GatewayServer.GatewayServerBuilder()
+      .authToken(secret)
+      .javaPort(0)
+      .javaAddress(localhost)
+      .callbackClient(py4j.GatewayServer.DEFAULT_PYTHON_PORT, localhost, secret)
+      .build()
     val thread = new Thread(new Runnable() {
       override def run(): Unit = Utils.logUncaughtExceptions {
         gatewayServer.start()
@@ -82,6 +89,7 @@ object PythonRunner {
     // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u:
     env.put("PYTHONUNBUFFERED", "YES") // value is needed to be set to a non-empty string
     env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort)
+    env.put("PYSPARK_GATEWAY_SECRET", secret)
     // pass conf spark.pyspark.python to python process, the only way to pass info to
     // python process is through environment variable.
     sparkConf.get(PYSPARK_PYTHON).foreach(env.put("PYSPARK_PYTHON", _))

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/internal/config/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 6bb98c3..82f0a04 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -352,6 +352,11 @@ package object config {
       .regexConf
       .createOptional
 
+  private[spark] val AUTH_SECRET_BIT_LENGTH =
+    ConfigBuilder("spark.authenticate.secretBitLength")
+      .intConf
+      .createWithDefault(256)
+
   private[spark] val NETWORK_AUTH_ENABLED =
     ConfigBuilder("spark.authenticate")
       .booleanConf

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
new file mode 100644
index 0000000..d15e793
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.security
+
+import java.io.{DataInputStream, DataOutputStream, InputStream}
+import java.net.Socket
+import java.nio.charset.StandardCharsets.UTF_8
+
+import org.apache.spark.SparkConf
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.util.Utils
+
+/**
+ * A class that can be used to add a simple authentication protocol to socket-based communication.
+ *
+ * The protocol is simple: an auth secret is written to the socket, and the other side checks the
+ * secret and writes either "ok" or "err" to the output. If authentication fails, the socket is
+ * not expected to be valid anymore.
+ *
+ * There's no secrecy, so this relies on the sockets being either local or somehow encrypted.
+ */
+private[spark] class SocketAuthHelper(conf: SparkConf) {
+
+  val secret = Utils.createSecret(conf)
+
+  /**
+   * Read the auth secret from the socket and compare to the expected value. Write the reply back
+   * to the socket.
+   *
+   * If authentication fails, this method will close the socket.
+   *
+   * @param s The client socket.
+   * @throws IllegalArgumentException If authentication fails.
+   */
+  def authClient(s: Socket): Unit = {
+    // Set the socket timeout while checking the auth secret. Reset it before returning.
+    val currentTimeout = s.getSoTimeout()
+    try {
+      s.setSoTimeout(10000)
+      val clientSecret = readUtf8(s)
+      if (secret == clientSecret) {
+        writeUtf8("ok", s)
+      } else {
+        writeUtf8("err", s)
+        JavaUtils.closeQuietly(s)
+      }
+    } finally {
+      s.setSoTimeout(currentTimeout)
+    }
+  }
+
+  /**
+   * Authenticate with a server by writing the auth secret and checking the server's reply.
+   *
+   * If authentication fails, this method will close the socket.
+   *
+   * @param s The socket connected to the server.
+   * @throws IllegalArgumentException If authentication fails.
+   */
+  def authToServer(s: Socket): Unit = {
+    writeUtf8(secret, s)
+
+    val reply = readUtf8(s)
+    if (reply != "ok") {
+      JavaUtils.closeQuietly(s)
+      throw new IllegalArgumentException("Authentication failed.")
+    }
+  }
+
+  protected def readUtf8(s: Socket): String = {
+    val din = new DataInputStream(s.getInputStream())
+    val len = din.readInt()
+    val bytes = new Array[Byte](len)
+    din.readFully(bytes)
+    new String(bytes, UTF_8)
+  }
+
+  protected def writeUtf8(str: String, s: Socket): Unit = {
+    val bytes = str.getBytes(UTF_8)
+    val dout = new DataOutputStream(s.getOutputStream())
+    dout.writeInt(bytes.length)
+    dout.write(bytes, 0, bytes.length)
+    dout.flush()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 dcad1b9..13adaa9 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.util
 
 import java.io._
+import java.lang.{Byte => JByte}
 import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
 import java.lang.reflect.InvocationTargetException
 import java.math.{MathContext, RoundingMode}
@@ -26,11 +27,11 @@ import java.nio.ByteBuffer
 import java.nio.channels.{Channels, FileChannel}
 import java.nio.charset.StandardCharsets
 import java.nio.file.Files
+import java.security.SecureRandom
 import java.util.{Locale, Properties, Random, UUID}
 import java.util.concurrent._
 import java.util.concurrent.atomic.AtomicBoolean
 import java.util.zip.GZIPInputStream
-import javax.net.ssl.HttpsURLConnection
 
 import scala.annotation.tailrec
 import scala.collection.JavaConverters._
@@ -44,6 +45,7 @@ import scala.util.matching.Regex
 
 import _root_.io.netty.channel.unix.Errors.NativeIoException
 import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
+import com.google.common.hash.HashCodes
 import com.google.common.io.{ByteStreams, Files => GFiles}
 import com.google.common.net.InetAddresses
 import org.apache.commons.lang3.SystemUtils
@@ -2704,6 +2706,15 @@ private[spark] object Utils extends Logging {
   def substituteAppId(opt: String, appId: String): String = {
     opt.replace("{{APP_ID}}", appId)
   }
+
+  def createSecret(conf: SparkConf): String = {
+    val bits = conf.get(AUTH_SECRET_BIT_LENGTH)
+    val rnd = new SecureRandom()
+    val secretBytes = new Array[Byte](bits / JByte.SIZE)
+    rnd.nextBytes(secretBytes)
+    HashCodes.fromBytes(secretBytes).toString()
+  }
+
 }
 
 private[util] object CallerContext extends Logging {

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala b/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala
new file mode 100644
index 0000000..e57cb70
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/security/SocketAuthHelperSuite.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.security
+
+import java.io.Closeable
+import java.net._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.internal.config._
+import org.apache.spark.util.Utils
+
+class SocketAuthHelperSuite extends SparkFunSuite {
+
+  private val conf = new SparkConf()
+  private val authHelper = new SocketAuthHelper(conf)
+
+  test("successful auth") {
+    Utils.tryWithResource(new ServerThread()) { server =>
+      Utils.tryWithResource(server.createClient()) { client =>
+        authHelper.authToServer(client)
+        server.close()
+        server.join()
+        assert(server.error == null)
+        assert(server.authenticated)
+      }
+    }
+  }
+
+  test("failed auth") {
+    Utils.tryWithResource(new ServerThread()) { server =>
+      Utils.tryWithResource(server.createClient()) { client =>
+        val badHelper = new SocketAuthHelper(new SparkConf().set(AUTH_SECRET_BIT_LENGTH, 128))
+        intercept[IllegalArgumentException] {
+          badHelper.authToServer(client)
+        }
+        server.close()
+        server.join()
+        assert(server.error != null)
+        assert(!server.authenticated)
+      }
+    }
+  }
+
+  private class ServerThread extends Thread with Closeable {
+
+    private val ss = new ServerSocket()
+    ss.bind(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0))
+
+    @volatile var error: Exception = _
+    @volatile var authenticated = false
+
+    setDaemon(true)
+    start()
+
+    def createClient(): Socket = {
+      new Socket(InetAddress.getLoopbackAddress(), ss.getLocalPort())
+    }
+
+    override def run(): Unit = {
+      var clientConn: Socket = null
+      try {
+        clientConn = ss.accept()
+        authHelper.authClient(clientConn)
+        authenticated = true
+      } catch {
+        case e: Exception =>
+          error = e
+      } finally {
+        Option(clientConn).foreach(_.close())
+      }
+    }
+
+    override def close(): Unit = {
+      try {
+        ss.close()
+      } finally {
+        interrupt()
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/dev/deps/spark-deps-hadoop-2.6
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index f479c13..f552b81 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -170,7 +170,7 @@ parquet-hadoop-1.10.0.jar
 parquet-hadoop-bundle-1.6.0.jar
 parquet-jackson-1.10.0.jar
 protobuf-java-2.5.0.jar
-py4j-0.10.6.jar
+py4j-0.10.7.jar
 pyrolite-4.13.jar
 scala-compiler-2.11.8.jar
 scala-library-2.11.8.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/dev/deps/spark-deps-hadoop-2.7
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index e7c4599..024b1fc 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -171,7 +171,7 @@ parquet-hadoop-1.10.0.jar
 parquet-hadoop-bundle-1.6.0.jar
 parquet-jackson-1.10.0.jar
 protobuf-java-2.5.0.jar
-py4j-0.10.6.jar
+py4j-0.10.7.jar
 pyrolite-4.13.jar
 scala-compiler-2.11.8.jar
 scala-library-2.11.8.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/dev/deps/spark-deps-hadoop-3.1
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1
index 3447cd7..938de7b 100644
--- a/dev/deps/spark-deps-hadoop-3.1
+++ b/dev/deps/spark-deps-hadoop-3.1
@@ -189,7 +189,7 @@ parquet-hadoop-1.10.0.jar
 parquet-hadoop-bundle-1.6.0.jar
 parquet-jackson-1.10.0.jar
 protobuf-java-2.5.0.jar
-py4j-0.10.6.jar
+py4j-0.10.7.jar
 pyrolite-4.13.jar
 re2j-1.1.jar
 scala-compiler-2.11.8.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/dev/run-pip-tests
----------------------------------------------------------------------
diff --git a/dev/run-pip-tests b/dev/run-pip-tests
index 1321c2b..7271d10 100755
--- a/dev/run-pip-tests
+++ b/dev/run-pip-tests
@@ -89,7 +89,7 @@ for python in "${PYTHON_EXECS[@]}"; do
       source "$VIRTUALENV_PATH"/bin/activate
     fi
     # Upgrade pip & friends if using virutal env
-    if [ ! -n "USE_CONDA" ]; then
+    if [ ! -n "$USE_CONDA" ]; then
       pip install --upgrade pip pypandoc wheel numpy
     fi
 

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/README.md
----------------------------------------------------------------------
diff --git a/python/README.md b/python/README.md
index 2e0112d..c020d84 100644
--- a/python/README.md
+++ b/python/README.md
@@ -29,4 +29,4 @@ The Python packaging for Spark is not intended to replace all of the other use c
 
 ## Python Requirements
 
-At its core PySpark depends on Py4J (currently version 0.10.6), but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow).
+At its core PySpark depends on Py4J (currently version 0.10.7), but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow).

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/docs/Makefile
----------------------------------------------------------------------
diff --git a/python/docs/Makefile b/python/docs/Makefile
index 09898f2..b8e0794 100644
--- a/python/docs/Makefile
+++ b/python/docs/Makefile
@@ -7,7 +7,7 @@ SPHINXBUILD   ?= sphinx-build
 PAPER         ?=
 BUILDDIR      ?= _build
 
-export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.6-src.zip)
+export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.7-src.zip)
 
 # User-friendly check for sphinx-build
 ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1)

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/lib/py4j-0.10.6-src.zip
----------------------------------------------------------------------
diff --git a/python/lib/py4j-0.10.6-src.zip b/python/lib/py4j-0.10.6-src.zip
deleted file mode 100644
index 2f8edcc..0000000
Binary files a/python/lib/py4j-0.10.6-src.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/lib/py4j-0.10.7-src.zip
----------------------------------------------------------------------
diff --git a/python/lib/py4j-0.10.7-src.zip b/python/lib/py4j-0.10.7-src.zip
new file mode 100644
index 0000000..128e321
Binary files /dev/null and b/python/lib/py4j-0.10.7-src.zip differ

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 7c66496..dbb463f 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -998,8 +998,8 @@ class SparkContext(object):
         # by runJob() in order to avoid having to pass a Python lambda into
         # SparkContext#runJob.
         mappedRDD = rdd.mapPartitions(partitionFunc)
-        port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, partitions)
-        return list(_load_from_socket(port, mappedRDD._jrdd_deserializer))
+        sock_info = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, partitions)
+        return list(_load_from_socket(sock_info, mappedRDD._jrdd_deserializer))
 
     def show_profiles(self):
         """ Print the profile stats to stdout """

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/daemon.py
----------------------------------------------------------------------
diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py
index 7bed521..ebdd665 100644
--- a/python/pyspark/daemon.py
+++ b/python/pyspark/daemon.py
@@ -29,7 +29,7 @@ from socket import AF_INET, SOCK_STREAM, SOMAXCONN
 from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT
 
 from pyspark.worker import main as worker_main
-from pyspark.serializers import read_int, write_int
+from pyspark.serializers import read_int, write_int, write_with_length, UTF8Deserializer
 
 
 def compute_real_exit_code(exit_code):
@@ -40,7 +40,7 @@ def compute_real_exit_code(exit_code):
         return 1
 
 
-def worker(sock):
+def worker(sock, authenticated):
     """
     Called by a worker process after the fork().
     """
@@ -56,6 +56,18 @@ def worker(sock):
     # otherwise writes also cause a seek that makes us miss data on the read side.
     infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536)
     outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536)
+
+    if not authenticated:
+        client_secret = UTF8Deserializer().loads(infile)
+        if os.environ["PYTHON_WORKER_FACTORY_SECRET"] == client_secret:
+            write_with_length("ok".encode("utf-8"), outfile)
+            outfile.flush()
+        else:
+            write_with_length("err".encode("utf-8"), outfile)
+            outfile.flush()
+            sock.close()
+            return 1
+
     exit_code = 0
     try:
         worker_main(infile, outfile)
@@ -153,8 +165,11 @@ def manager():
                         write_int(os.getpid(), outfile)
                         outfile.flush()
                         outfile.close()
+                        authenticated = False
                         while True:
-                            code = worker(sock)
+                            code = worker(sock, authenticated)
+                            if code == 0:
+                                authenticated = True
                             if not reuse or code:
                                 # wait for closing
                                 try:

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 3e704fe..0afbe9d 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -21,16 +21,19 @@ import sys
 import select
 import signal
 import shlex
+import shutil
 import socket
 import platform
+import tempfile
+import time
 from subprocess import Popen, PIPE
 
 if sys.version >= '3':
     xrange = range
 
-from py4j.java_gateway import java_import, JavaGateway, GatewayClient
+from py4j.java_gateway import java_import, JavaGateway, GatewayParameters
 from pyspark.find_spark_home import _find_spark_home
-from pyspark.serializers import read_int
+from pyspark.serializers import read_int, write_with_length, UTF8Deserializer
 
 
 def launch_gateway(conf=None):
@@ -41,6 +44,7 @@ def launch_gateway(conf=None):
     """
     if "PYSPARK_GATEWAY_PORT" in os.environ:
         gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"])
+        gateway_secret = os.environ["PYSPARK_GATEWAY_SECRET"]
     else:
         SPARK_HOME = _find_spark_home()
         # Launch the Py4j gateway using Spark's run command so that we pick up the
@@ -59,40 +63,40 @@ def launch_gateway(conf=None):
             ])
         command = command + shlex.split(submit_args)
 
-        # Start a socket that will be used by PythonGatewayServer to communicate its port to us
-        callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
-        callback_socket.bind(('127.0.0.1', 0))
-        callback_socket.listen(1)
-        callback_host, callback_port = callback_socket.getsockname()
-        env = dict(os.environ)
-        env['_PYSPARK_DRIVER_CALLBACK_HOST'] = callback_host
-        env['_PYSPARK_DRIVER_CALLBACK_PORT'] = str(callback_port)
-
-        # Launch the Java gateway.
-        # We open a pipe to stdin so that the Java gateway can die when the pipe is broken
-        if not on_windows:
-            # Don't send ctrl-c / SIGINT to the Java gateway:
-            def preexec_func():
-                signal.signal(signal.SIGINT, signal.SIG_IGN)
-            proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env)
-        else:
-            # preexec_fn not supported on Windows
-            proc = Popen(command, stdin=PIPE, env=env)
-
-        gateway_port = None
-        # We use select() here in order to avoid blocking indefinitely if the subprocess dies
-        # before connecting
-        while gateway_port is None and proc.poll() is None:
-            timeout = 1  # (seconds)
-            readable, _, _ = select.select([callback_socket], [], [], timeout)
-            if callback_socket in readable:
-                gateway_connection = callback_socket.accept()[0]
-                # Determine which ephemeral port the server started on:
-                gateway_port = read_int(gateway_connection.makefile(mode="rb"))
-                gateway_connection.close()
-                callback_socket.close()
-        if gateway_port is None:
-            raise Exception("Java gateway process exited before sending the driver its port number")
+        # Create a temporary directory where the gateway server should write the connection
+        # information.
+        conn_info_dir = tempfile.mkdtemp()
+        try:
+            fd, conn_info_file = tempfile.mkstemp(dir=conn_info_dir)
+            os.close(fd)
+            os.unlink(conn_info_file)
+
+            env = dict(os.environ)
+            env["_PYSPARK_DRIVER_CONN_INFO_PATH"] = conn_info_file
+
+            # Launch the Java gateway.
+            # We open a pipe to stdin so that the Java gateway can die when the pipe is broken
+            if not on_windows:
+                # Don't send ctrl-c / SIGINT to the Java gateway:
+                def preexec_func():
+                    signal.signal(signal.SIGINT, signal.SIG_IGN)
+                proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env)
+            else:
+                # preexec_fn not supported on Windows
+                proc = Popen(command, stdin=PIPE, env=env)
+
+            # Wait for the file to appear, or for the process to exit, whichever happens first.
+            while not proc.poll() and not os.path.isfile(conn_info_file):
+                time.sleep(0.1)
+
+            if not os.path.isfile(conn_info_file):
+                raise Exception("Java gateway process exited before sending its port number")
+
+            with open(conn_info_file, "rb") as info:
+                gateway_port = read_int(info)
+                gateway_secret = UTF8Deserializer().loads(info)
+        finally:
+            shutil.rmtree(conn_info_dir)
 
         # In Windows, ensure the Java child processes do not linger after Python has exited.
         # In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when
@@ -111,7 +115,9 @@ def launch_gateway(conf=None):
             atexit.register(killChild)
 
     # Connect to the gateway
-    gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=True)
+    gateway = JavaGateway(
+        gateway_parameters=GatewayParameters(port=gateway_port, auth_token=gateway_secret,
+                                             auto_convert=True))
 
     # Import the classes used by PySpark
     java_import(gateway.jvm, "org.apache.spark.SparkConf")
@@ -126,3 +132,16 @@ def launch_gateway(conf=None):
     java_import(gateway.jvm, "scala.Tuple2")
 
     return gateway
+
+
+def do_server_auth(conn, auth_secret):
+    """
+    Performs the authentication protocol defined by the SocketAuthHelper class on the given
+    file-like object 'conn'.
+    """
+    write_with_length(auth_secret.encode("utf-8"), conn)
+    conn.flush()
+    reply = UTF8Deserializer().loads(conn)
+    if reply != "ok":
+        conn.close()
+        raise Exception("Unexpected reply from iterator server.")

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/rdd.py
----------------------------------------------------------------------
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 4b44f76..d5a237a 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -39,9 +39,11 @@ if sys.version > '3':
 else:
     from itertools import imap as map, ifilter as filter
 
+from pyspark.java_gateway import do_server_auth
 from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
     BatchedSerializer, CloudPickleSerializer, PairDeserializer, \
-    PickleSerializer, pack_long, AutoBatchedSerializer
+    PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \
+    UTF8Deserializer
 from pyspark.join import python_join, python_left_outer_join, \
     python_right_outer_join, python_full_outer_join, python_cogroup
 from pyspark.statcounter import StatCounter
@@ -136,7 +138,8 @@ def _parse_memory(s):
     return int(float(s[:-1]) * units[s[-1].lower()])
 
 
-def _load_from_socket(port, serializer):
+def _load_from_socket(sock_info, serializer):
+    port, auth_secret = sock_info
     sock = None
     # Support for both IPv4 and IPv6.
     # On most of IPv6-ready systems, IPv6 will take precedence.
@@ -156,8 +159,12 @@ def _load_from_socket(port, serializer):
     # The RDD materialization time is unpredicable, if we set a timeout for socket reading
     # operation, it will very possibly fail. See SPARK-18281.
     sock.settimeout(None)
+
+    sockfile = sock.makefile("rwb", 65536)
+    do_server_auth(sockfile, auth_secret)
+
     # The socket will be automatically closed when garbage-collected.
-    return serializer.load_stream(sock.makefile("rb", 65536))
+    return serializer.load_stream(sockfile)
 
 
 def ignore_unicode_prefix(f):
@@ -822,8 +829,8 @@ class RDD(object):
             to be small, as all the data is loaded into the driver's memory.
         """
         with SCCallSiteSync(self.context) as css:
-            port = self.ctx._jvm.PythonRDD.collectAndServe(self._jrdd.rdd())
-        return list(_load_from_socket(port, self._jrdd_deserializer))
+            sock_info = self.ctx._jvm.PythonRDD.collectAndServe(self._jrdd.rdd())
+        return list(_load_from_socket(sock_info, self._jrdd_deserializer))
 
     def reduce(self, f):
         """
@@ -2380,8 +2387,8 @@ class RDD(object):
         [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
         """
         with SCCallSiteSync(self.context) as css:
-            port = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe(self._jrdd.rdd())
-        return _load_from_socket(port, self._jrdd_deserializer)
+            sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe(self._jrdd.rdd())
+        return _load_from_socket(sock_info, self._jrdd_deserializer)
 
 
 def _prepare_for_python_RDD(sc, command):

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/sql/dataframe.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 16f8e52..213dc15 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -463,8 +463,8 @@ class DataFrame(object):
         [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
         """
         with SCCallSiteSync(self._sc) as css:
-            port = self._jdf.collectToPython()
-        return list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
+            sock_info = self._jdf.collectToPython()
+        return list(_load_from_socket(sock_info, BatchedSerializer(PickleSerializer())))
 
     @ignore_unicode_prefix
     @since(2.0)
@@ -477,8 +477,8 @@ class DataFrame(object):
         [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
         """
         with SCCallSiteSync(self._sc) as css:
-            port = self._jdf.toPythonIterator()
-        return _load_from_socket(port, BatchedSerializer(PickleSerializer()))
+            sock_info = self._jdf.toPythonIterator()
+        return _load_from_socket(sock_info, BatchedSerializer(PickleSerializer()))
 
     @ignore_unicode_prefix
     @since(1.3)
@@ -2087,8 +2087,8 @@ class DataFrame(object):
         .. note:: Experimental.
         """
         with SCCallSiteSync(self._sc) as css:
-            port = self._jdf.collectAsArrowToPython()
-        return list(_load_from_socket(port, ArrowSerializer()))
+            sock_info = self._jdf.collectAsArrowToPython()
+        return list(_load_from_socket(sock_info, ArrowSerializer()))
 
     ##########################################################################################
     # Pandas compatibility

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/pyspark/worker.py
----------------------------------------------------------------------
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index a1a4336..8bb63fc 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -27,6 +27,7 @@ import traceback
 
 from pyspark.accumulators import _accumulatorRegistry
 from pyspark.broadcast import Broadcast, _broadcastRegistry
+from pyspark.java_gateway import do_server_auth
 from pyspark.taskcontext import TaskContext
 from pyspark.files import SparkFiles
 from pyspark.rdd import PythonEvalType
@@ -301,9 +302,11 @@ def main(infile, outfile):
 
 
 if __name__ == '__main__':
-    # Read a local port to connect to from stdin
-    java_port = int(sys.stdin.readline())
+    # Read information about how to connect back to the JVM from the environment.
+    java_port = int(os.environ["PYTHON_WORKER_FACTORY_PORT"])
+    auth_secret = os.environ["PYTHON_WORKER_FACTORY_SECRET"]
     sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
     sock.connect(("127.0.0.1", java_port))
     sock_file = sock.makefile("rwb", 65536)
+    do_server_auth(sock_file, auth_secret)
     main(sock_file, sock_file)

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/python/setup.py
----------------------------------------------------------------------
diff --git a/python/setup.py b/python/setup.py
index 794cece..d309e05 100644
--- a/python/setup.py
+++ b/python/setup.py
@@ -201,7 +201,7 @@ try:
             'pyspark.examples.src.main.python': ['*.py', '*/*.py']},
         scripts=scripts,
         license='http://www.apache.org/licenses/LICENSE-2.0',
-        install_requires=['py4j==0.10.6'],
+        install_requires=['py4j==0.10.7'],
         setup_requires=['pypandoc'],
         extras_require={
             'ml': ['numpy>=1.7'],

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index bafb129..134b3e5 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -1152,7 +1152,7 @@ private[spark] class Client(
         val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
         require(pyArchivesFile.exists(),
           s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.")
-        val py4jFile = new File(pyLibPath, "py4j-0.10.6-src.zip")
+        val py4jFile = new File(pyLibPath, "py4j-0.10.7-src.zip")
         require(py4jFile.exists(),
           s"$py4jFile not found; cannot run pyspark application in YARN mode.")
         Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath())

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
----------------------------------------------------------------------
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index a129be7..59b0f29 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -265,7 +265,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
     // needed locations.
     val sparkHome = sys.props("spark.test.home")
     val pythonPath = Seq(
-        s"$sparkHome/python/lib/py4j-0.10.6-src.zip",
+        s"$sparkHome/python/lib/py4j-0.10.7-src.zip",
         s"$sparkHome/python")
     val extraEnvVars = Map(
       "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator),

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/sbin/spark-config.sh
----------------------------------------------------------------------
diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh
index bac154e..bf3da18 100755
--- a/sbin/spark-config.sh
+++ b/sbin/spark-config.sh
@@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}"
 # Add the PySpark classes to the PYTHONPATH:
 if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then
   export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}"
-  export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:${PYTHONPATH}"
+  export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-src.zip:${PYTHONPATH}"
   export PYSPARK_PYTHONPATH_SET=1
 fi

http://git-wip-us.apache.org/repos/asf/spark/blob/cc613b55/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index cd4def7..d518e07 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -3187,7 +3187,7 @@ class Dataset[T] private[sql](
     EvaluatePython.javaToPython(rdd)
   }
 
-  private[sql] def collectToPython(): Int = {
+  private[sql] def collectToPython(): Array[Any] = {
     EvaluatePython.registerPicklers()
     withAction("collectToPython", queryExecution) { plan =>
       val toJava: (Any) => Any = EvaluatePython.toJava(_, schema)
@@ -3200,7 +3200,7 @@ class Dataset[T] private[sql](
   /**
    * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    */
-  private[sql] def collectAsArrowToPython(): Int = {
+  private[sql] def collectAsArrowToPython(): Array[Any] = {
     withAction("collectAsArrowToPython", queryExecution) { plan =>
       val iter: Iterator[Array[Byte]] =
         toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
@@ -3208,7 +3208,7 @@ class Dataset[T] private[sql](
     }
   }
 
-  private[sql] def toPythonIterator(): Int = {
+  private[sql] def toPythonIterator(): Array[Any] = {
     withNewExecutionId {
       PythonRDD.toLocalIteratorAndServe(javaToPython.rdd)
     }


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