-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-33143][PYTHON] Add configurable timeout to python server and client #30389
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 4 commits
09b1235
17d357b
424be64
d9feed8
6e8e194
f504af3
1595581
a67acd7
0f9e587
cd2d595
d9f0a1b
2913fb1
ef137b6
363f3bb
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 |
|---|---|---|
|
|
@@ -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_GATEWAY_CONNECT_TIMEOUT | ||
| import org.apache.spark.network.util.JavaUtils | ||
| import org.apache.spark.util.{ThreadUtils, Utils} | ||
|
|
||
|
|
@@ -34,29 +36,35 @@ 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 configured seconds | ||
|
||
| val timeout = SparkEnv.get.conf.get(PYTHON_GATEWAY_CONNECT_TIMEOUT).toInt | ||
| 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 port ${sock.getLocalPort}") | ||
| authHelper.authClient(sock) | ||
| logTrace("Client authenticated") | ||
| promise.complete(Try(handleConnection(sock))) | ||
| } finally { | ||
| JavaUtils.closeQuietly(serverSocket) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,15 +30,18 @@ import org.apache.hadoop.conf.Configuration | |
| import org.apache.hadoop.io.{LongWritable, Text} | ||
| import org.apache.hadoop.mapred.TextInputFormat | ||
| import org.apache.hadoop.mapreduce.lib.input.FileInputFormat | ||
| import org.mockito.Mockito.mock | ||
|
|
||
| import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} | ||
| import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite} | ||
| import org.apache.spark.api.java.JavaSparkContext | ||
| import org.apache.spark.rdd.{HadoopRDD, RDD} | ||
| import org.apache.spark.security.{SocketAuthHelper, SocketAuthServer} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| class PythonRDDSuite extends SparkFunSuite with LocalSparkContext { | ||
|
|
||
| private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) | ||
|
|
||
| var tempDir: File = _ | ||
|
|
||
| override def beforeAll(): Unit = { | ||
|
|
@@ -76,12 +79,22 @@ class PythonRDDSuite extends SparkFunSuite with LocalSparkContext { | |
| } | ||
|
|
||
| test("python server error handling") { | ||
| val authHelper = new SocketAuthHelper(new SparkConf()) | ||
| val errorServer = new ExceptionPythonServer(authHelper) | ||
| val client = new Socket(InetAddress.getLoopbackAddress(), errorServer.port) | ||
| authHelper.authToServer(client) | ||
| val ex = intercept[Exception] { errorServer.getResult(Duration(1, "second")) } | ||
| assert(ex.getCause().getMessage().contains("exception within handleConnection")) | ||
| val savedSparkEnv = SparkEnv.get | ||
| try { | ||
| val conf = new SparkConf() | ||
| val env = mock(classOf[SparkEnv]) | ||
| doReturn(conf).when(env).conf | ||
| SparkEnv.set(env) | ||
|
||
|
|
||
| val authHelper = new SocketAuthHelper(conf) | ||
| val errorServer = new ExceptionPythonServer(authHelper) | ||
| val client = new Socket(InetAddress.getLoopbackAddress(), errorServer.port) | ||
| authHelper.authToServer(client) | ||
| val ex = intercept[Exception] { errorServer.getResult(Duration(1, "second")) } | ||
| assert(ex.getCause().getMessage().contains("exception within handleConnection")) | ||
| } finally { | ||
| SparkEnv.set(savedSparkEnv) | ||
| } | ||
| } | ||
|
|
||
| class ExceptionPythonServer(authHelper: SocketAuthHelper) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.