utkarsh39 opened a new pull request, #42385:
URL: https://github.com/apache/spark/pull/42385
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR:
https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g.,
'[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a
faster review.
7. If you want to add a new configuration, please read the guideline first
for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
8. If you want to add or modify an error type or message, please read the
guideline first in
'core/src/main/resources/error/README.md'.
-->
### What changes were proposed in this pull request?
PythonRunner, a utility that executes Python UDFs in Spark, uses two threads
in a producer-consumer model today. This multi-threading model is problematic
and confusing as Spark's execution model within a task is commonly understood
to be single-threaded.
More importantly, this departure of a double-threaded execution resulted in
a series of customer issues involving [race
conditions](https://issues.apache.org/jira/browse/SPARK-33277) and
[deadlocks](https://issues.apache.org/jira/browse/SPARK-38677) between threads
as the code was hard to reason about. There have been multiple attempts to
reign in these issues, viz., [fix
1](https://issues.apache.org/jira/browse/SPARK-22535), [fix
2](https://github.com/apache/spark/pull/30177), [fix
3](https://github.com/apache/spark/commit/243c321db2f02f6b4d926114bd37a6e74c2be185).
Moreover, the fixes have made the code base somewhat abstruse by introducing
multiple daemon [monitor
threads](https://github.com/apache/spark/blob/a3a32912be04d3760cb34eb4b79d6d481bbec502/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala#L579)
to detect deadlocks. This PR makes PythonRunner single-threaded making it
easier to reason about and improving code health.
#### Current Execution Model in Spark for Python UDFs
For queries containing Python UDFs, the main Java task thread spins up a new
writer thread to pipe data from the child Spark plan into the Python worker
evaluating the UDF. The writer thread runs in a tight loop: evaluates the child
Spark plan, and feeds the resulting output to the Python worker. The main task
thread simultaneously consumes the Python UDF’s output and evaluates the parent
Spark plan to produce the final result.
The I/O to/from the Python worker uses blocking Java Sockets necessitating
the use of two threads, one responsible for input to the Python worker and the
other for output. Without two threads, it is easy to run into a deadlock. For
example, the task can block forever waiting for the output from the Python
worker. The output will never arrive until the input is supplied to the Python
worker, which is not possible as the task thread is blocked while waiting on
output.
#### Proposed Fix
The proposed fix is to move to the standard single-threaded execution model
within a task, i.e., to do away with the writer thread. In addition to
mitigating the crashes, the fix reduces the complexity of the existing code by
doing away with many safety checks in place to track deadlocks in the
double-threaded execution model.
In the new model, the main task thread alternates between consuming/feeding
data to the Python worker using asynchronous I/O through Java’s
[SocketChannel](https://docs.oracle.com/javase/7/docs/api/java/nio/channels/SocketChannel.html).
See the `read()` method in the code below for approximately how this is
achieved.
```
case class PythonUDFRunner {
private var nextRow: Row = _
private var endOfStream = false
private var childHasNext = true
private var buffer: ByteBuffer = _
def hasNext(): Boolean = nextRow != null || {
if (!endOfStream) {
read(buffer)
nextRow = deserialize(buffer)
hasNext
} else {
false
}
}
def next(): Row = {
if (hasNext) {
val outputRow = nextRow
nextRow = null
outputRow
} else {
null
}
}
def read(buf: Array[Byte]): Row = {
var n = 0
while (n == 0) {
// Alternate between reading/writing to the Python worker using async I/O
if (pythonWorker.isReadable) {
n = pythonWorker.read(buf)
}
if (pythonWorker.isWritable) {
consumeChildPlanAndWriteDataToPythonWorker()
}
}
def consumeChildPlanAndWriteDataToPythonWorker(): Unit = {
// Tracks whether the connection to the Python worker can be written
to.
var socketAcceptsInput = true
while (socketAcceptsInput && (childHasNext || buffer.hasRemaining)) {
if (!buffer.hasRemaining && childHasNext) {
// Consume data from the child and buffer it.
writeToBuffer(childPlan.next(), buffer)
childHasNext = childPlan.hasNext()
if (!childHasNext) {
// Exhausted child plan’s output. Write a keyword to the Python
worker signaling the end of data input.
writeToBuffer(endOfStream)
}
}
// Try to write as much buffered data as possible to the Python
worker.
while (buffer.hasRemaining && socketAcceptsInput) {
val n = writeToPythonWorker(buffer)
// `writeToPythonWorker()` returns 0 when the socket cannot accept
more data right now.
socketAcceptsInput = n > 0
}
}
}
}
```
### Why are the changes needed?
This PR makes PythonRunner single-threaded making it easier to reason about
and improving code health.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing tests.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]