Skip to content
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

[SPARK-33143][PYTHON] Add configurable timeout to python server and client #30389

Closed
wants to merge 14 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](

private val conf = SparkEnv.get.conf
protected val bufferSize: Int = conf.get(BUFFER_SIZE)
protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
private val reuseWorker = conf.get(PYTHON_WORKER_REUSE)

// All the Python functions should have the same exec, version and envvars.
Expand Down Expand Up @@ -140,6 +141,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
if (workerMemoryMb.isDefined) {
envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString)
}
envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap)
// Whether is the worker released into idle pool or closed. When any codes try to release or
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,4 +85,8 @@ private[spark] object PythonUtils {
def getBroadcastThreshold(sc: JavaSparkContext): Long = {
sc.conf.get(org.apache.spark.internal.config.BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD)
}

def getPythonAuthSocketTimeout(sc: JavaSparkContext): Long = {
sc.conf.get(org.apache.spark.internal.config.Python.PYTHON_AUTH_SOCKET_TIMEOUT)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,4 +50,10 @@ private[spark] object Python {
.version("2.4.0")
.bytesConf(ByteUnit.MiB)
.createOptional

val PYTHON_AUTH_SOCKET_TIMEOUT = ConfigBuilder("spark.python.authenticate.socketTimeout")
.internal()
.version("3.1.0")
gaborgsomogyi marked this conversation as resolved.
Show resolved Hide resolved
.timeConf(TimeUnit.SECONDS)
.createWithDefaultString("15s")
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils
*
* There's no secrecy, so this relies on the sockets being either local or somehow encrypted.
*/
private[spark] class SocketAuthHelper(conf: SparkConf) {
private[spark] class SocketAuthHelper(val conf: SparkConf) {

val secret = Utils.createSecret(conf)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import scala.concurrent.duration.Duration
import scala.util.Try

import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.Python.PYTHON_AUTH_SOCKET_TIMEOUT
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.util.{ThreadUtils, Utils}

Expand All @@ -34,31 +36,38 @@ import org.apache.spark.util.{ThreadUtils, Utils}
* handling one batch of data, with authentication and error handling.
*
* The socket server can only accept one connection, or close if no connection
* in 15 seconds.
* in configurable amount of seconds (default 15).
*/
private[spark] abstract class SocketAuthServer[T](
authHelper: SocketAuthHelper,
threadName: String) {
threadName: String) extends Logging {

def this(env: SparkEnv, threadName: String) = this(new SocketAuthHelper(env.conf), threadName)
def this(threadName: String) = this(SparkEnv.get, threadName)

private val promise = Promise[T]()

private def startServer(): (Int, String) = {
logTrace("Creating listening socket")
val serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1)))
// Close the socket if no connection in 15 seconds
serverSocket.setSoTimeout(15000)
// Close the socket if no connection in the configured seconds
val timeout = authHelper.conf.get(PYTHON_AUTH_SOCKET_TIMEOUT).toInt
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, actually the test in PythonRDDSuite was related.

In python server error handling test we don't initialize SparkEnv because we don't create SparkContext or executor there by default, but SparkEnv.get was used in the previous change which ended up with NPE.

Now, I made a change to work around by using SparkConf directly here.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Got it~

logTrace(s"Setting timeout to $timeout sec")
serverSocket.setSoTimeout(timeout * 1000)

new Thread(threadName) {
setDaemon(true)
override def run(): Unit = {
var sock: Socket = null
try {
logTrace(s"Waiting for connection on port ${serverSocket.getLocalPort}")
sock = serverSocket.accept()
logTrace(s"Connection accepted from address ${sock.getRemoteSocketAddress}")
authHelper.authClient(sock)
logTrace("Client authenticated")
promise.complete(Try(handleConnection(sock)))
} finally {
logTrace("Closing server")
JavaUtils.closeQuietly(serverSocket)
JavaUtils.closeQuietly(sock)
}
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,8 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize,
# data via a socket.
# scala's mangled names w/ $ in them require special treatment.
self._encryption_enabled = self._jvm.PythonUtils.isEncryptionEnabled(self._jsc)
os.environ["SPARK_AUTH_SOCKET_TIMEOUT"] = \
str(self._jvm.PythonUtils.getPythonAuthSocketTimeout(self._jsc))

self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
self.pythonVer = "%d.%d" % sys.version_info[:2]
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/java_gateway.py
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ def local_connect_and_auth(port, auth_secret):
af, socktype, proto, _, sa = res
try:
sock = socket.socket(af, socktype, proto)
sock.settimeout(15)
sock.settimeout(int(os.environ.get("SPARK_AUTH_SOCKET_TIMEOUT", 15)))
sock.connect(sa)
sockfile = sock.makefile("rwb", int(os.environ.get("SPARK_BUFFER_SIZE", 65536)))
_do_server_auth(sockfile, auth_secret)
Expand Down