-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-51688][PYTHON] Use Unix Domain Socket between Python and JVM communication #50466
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,9 +20,9 @@ package org.apache.spark.api.python | |
| import java.io._ | ||
| import java.net._ | ||
| import java.nio.ByteBuffer | ||
| import java.nio.channels.SelectionKey | ||
| import java.nio.charset.StandardCharsets.UTF_8 | ||
| import java.nio.channels.{AsynchronousCloseException, Channels, SelectionKey, ServerSocketChannel, SocketChannel} | ||
| import java.nio.file.{Files => JavaFiles, Path} | ||
| import java.util.UUID | ||
| import java.util.concurrent.{ConcurrentHashMap, TimeUnit} | ||
| import java.util.concurrent.atomic.AtomicBoolean | ||
|
|
||
|
|
@@ -201,9 +201,10 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| // Python accumulator is always set in production except in tests. See SPARK-27893 | ||
| private val maybeAccumulator: Option[PythonAccumulator] = Option(accumulator) | ||
|
|
||
| // Expose a ServerSocket to support method calls via socket from Python side. Only relevant for | ||
| // for tasks that are a part of barrier stage, refer [[BarrierTaskContext]] for details. | ||
| private[spark] var serverSocket: Option[ServerSocket] = None | ||
| // Expose a ServerSocketChannel to support method calls via socket from Python side. | ||
| // Only relevant for tasks that are a part of barrier stage, refer | ||
| // `BarrierTaskContext` for details. | ||
| private[spark] var serverSocketChannel: Option[ServerSocketChannel] = None | ||
|
|
||
| // Authentication helper used when serving method calls via socket from Python side. | ||
| private lazy val authHelper = new SocketAuthHelper(conf) | ||
|
|
@@ -347,6 +348,11 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| def writeNextInputToStream(dataOut: DataOutputStream): Boolean | ||
|
|
||
| def open(dataOut: DataOutputStream): Unit = Utils.logUncaughtExceptions { | ||
| val isUnixDomainSock = authHelper.conf.get(PYTHON_UNIX_DOMAIN_SOCKET_ENABLED) | ||
| lazy val sockPath = new File( | ||
| authHelper.conf.get(PYTHON_UNIX_DOMAIN_SOCKET_DIR) | ||
| .getOrElse(System.getProperty("java.io.tmpdir")), | ||
| s".${UUID.randomUUID()}.sock") | ||
| try { | ||
| // Partition index | ||
| dataOut.writeInt(partitionIndex) | ||
|
|
@@ -356,27 +362,34 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| // Init a ServerSocket to accept method calls from Python side. | ||
| val isBarrier = context.isInstanceOf[BarrierTaskContext] | ||
| if (isBarrier) { | ||
| serverSocket = Some(new ServerSocket(/* port */ 0, | ||
| /* backlog */ 1, | ||
| InetAddress.getByName("localhost"))) | ||
| // A call to accept() for ServerSocket shall block infinitely. | ||
| serverSocket.foreach(_.setSoTimeout(0)) | ||
| if (isUnixDomainSock) { | ||
| serverSocketChannel = Some(ServerSocketChannel.open(StandardProtocolFamily.UNIX)) | ||
| sockPath.deleteOnExit() | ||
| serverSocketChannel.get.bind(UnixDomainSocketAddress.of(sockPath.getPath)) | ||
|
||
| } else { | ||
| serverSocketChannel = Some(ServerSocketChannel.open()) | ||
| serverSocketChannel.foreach(_.bind( | ||
HyukjinKwon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 1)) | ||
| // A call to accept() for ServerSocket shall block infinitely. | ||
| serverSocketChannel.foreach(_.socket().setSoTimeout(0)) | ||
| } | ||
|
|
||
| new Thread("accept-connections") { | ||
| setDaemon(true) | ||
|
|
||
| override def run(): Unit = { | ||
| while (!serverSocket.get.isClosed()) { | ||
| var sock: Socket = null | ||
| while (serverSocketChannel.get.isOpen()) { | ||
| var sock: SocketChannel = null | ||
| try { | ||
| sock = serverSocket.get.accept() | ||
| sock = serverSocketChannel.get.accept() | ||
| // Wait for function call from python side. | ||
| sock.setSoTimeout(10000) | ||
| if (!isUnixDomainSock) sock.socket().setSoTimeout(10000) | ||
| authHelper.authClient(sock) | ||
| val input = new DataInputStream(sock.getInputStream()) | ||
| val input = new DataInputStream(Channels.newInputStream(sock)) | ||
| val requestMethod = input.readInt() | ||
| // The BarrierTaskContext function may wait infinitely, socket shall not timeout | ||
| // before the function finishes. | ||
| sock.setSoTimeout(0) | ||
| if (!isUnixDomainSock) sock.socket().setSoTimeout(0) | ||
| requestMethod match { | ||
| case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => | ||
| barrierAndServe(requestMethod, sock) | ||
|
|
@@ -385,13 +398,14 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| barrierAndServe(requestMethod, sock, message) | ||
| case _ => | ||
| val out = new DataOutputStream(new BufferedOutputStream( | ||
| sock.getOutputStream)) | ||
| Channels.newOutputStream(sock))) | ||
| writeUTF(BarrierTaskContextMessageProtocol.ERROR_UNRECOGNIZED_FUNCTION, out) | ||
| } | ||
| } catch { | ||
| case e: SocketException if e.getMessage.contains("Socket closed") => | ||
| // It is possible that the ServerSocket is not closed, but the native socket | ||
| // has already been closed, we shall catch and silently ignore this case. | ||
| case _: AsynchronousCloseException => | ||
| // Ignore to make less noisy. These will be closed when tasks | ||
| // are finished by listeners. | ||
| if (isUnixDomainSock) sockPath.delete() | ||
| } finally { | ||
| if (sock != null) { | ||
| sock.close() | ||
|
|
@@ -401,33 +415,35 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| } | ||
| }.start() | ||
| } | ||
| val secret = if (isBarrier) { | ||
| authHelper.secret | ||
| } else { | ||
| "" | ||
| } | ||
| if (isBarrier) { | ||
| // Close ServerSocket on task completion. | ||
| serverSocket.foreach { server => | ||
| context.addTaskCompletionListener[Unit](_ => server.close()) | ||
| serverSocketChannel.foreach { server => | ||
| context.addTaskCompletionListener[Unit] { _ => | ||
| server.close() | ||
| if (isUnixDomainSock) sockPath.delete() | ||
| } | ||
| } | ||
| val boundPort: Int = serverSocket.map(_.getLocalPort).getOrElse(0) | ||
| if (boundPort == -1) { | ||
| val message = "ServerSocket failed to bind to Java side." | ||
| logError(message) | ||
| throw new SparkException(message) | ||
HyukjinKwon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| if (isUnixDomainSock) { | ||
| logDebug(s"Started ServerSocket on with Unix Domain Socket $sockPath.") | ||
| dataOut.writeBoolean(/* isBarrier = */true) | ||
| dataOut.writeInt(-1) | ||
| PythonRDD.writeUTF(sockPath.getPath, dataOut) | ||
| } else { | ||
| val boundPort: Int = serverSocketChannel.map(_.socket().getLocalPort).getOrElse(-1) | ||
| if (boundPort == -1) { | ||
| val message = "ServerSocket failed to bind to Java side." | ||
| logError(message) | ||
| throw new SparkException(message) | ||
| } | ||
| logDebug(s"Started ServerSocket on port $boundPort.") | ||
| dataOut.writeBoolean(/* isBarrier = */true) | ||
| dataOut.writeInt(boundPort) | ||
| PythonRDD.writeUTF(authHelper.secret, dataOut) | ||
| } | ||
| logDebug(s"Started ServerSocket on port $boundPort.") | ||
| dataOut.writeBoolean(/* isBarrier = */true) | ||
| dataOut.writeInt(boundPort) | ||
| } else { | ||
| dataOut.writeBoolean(/* isBarrier = */false) | ||
| dataOut.writeInt(0) | ||
| } | ||
| // Write out the TaskContextInfo | ||
| val secretBytes = secret.getBytes(UTF_8) | ||
| dataOut.writeInt(secretBytes.length) | ||
| dataOut.write(secretBytes, 0, secretBytes.length) | ||
| dataOut.writeInt(context.stageId()) | ||
| dataOut.writeInt(context.partitionId()) | ||
| dataOut.writeInt(context.attemptNumber()) | ||
|
|
@@ -485,12 +501,12 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( | |
| /** | ||
| * Gateway to call BarrierTaskContext methods. | ||
| */ | ||
| def barrierAndServe(requestMethod: Int, sock: Socket, message: String = ""): Unit = { | ||
| def barrierAndServe(requestMethod: Int, sock: SocketChannel, message: String = ""): Unit = { | ||
| require( | ||
| serverSocket.isDefined, | ||
| serverSocketChannel.isDefined, | ||
| "No available ServerSocket to redirect the BarrierTaskContext method call." | ||
| ) | ||
| val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) | ||
| val out = new DataOutputStream(new BufferedOutputStream(Channels.newOutputStream(sock))) | ||
| try { | ||
| val messages = requestMethod match { | ||
| case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Instead of if/else's everywhere you could consider using an interface with two implementations. That improves readability of the code by a lot...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yup I will do it in a separate PR that factors those out. I think we can take the common code out to utils.