-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-42960] [CONNECT] [SS] Add await_termination() and exception() API for Streaming Query in Python #40785
Changes from 36 commits
4d0fcdd
0ae7e33
b14fe81
17720b7
1e68a3c
dc05be8
c1674eb
23b9c93
60ddd01
e576821
304d01e
26e2488
e25f7e6
aa1d4c2
e82678e
720e3d2
1f3ba94
d8a6e9f
0c2776a
da6fc21
ce8615f
7a377ec
be76438
e489501
7045f9d
83413c8
8e848c9
ad73db6
ea3bb35
dd4d54b
3b375cc
34c28e7
c487cd7
0c2133b
2384ba1
da1d3e1
eb19c2f
ed67070
566e9fc
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 |
---|---|---|
|
@@ -52,7 +52,7 @@ import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager | |
import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, UdfPacket} | ||
import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE | ||
import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry | ||
import org.apache.spark.sql.connect.service.SparkConnectStreamHandler | ||
import org.apache.spark.sql.connect.service.{SparkConnectService, SparkConnectStreamHandler} | ||
import org.apache.spark.sql.errors.QueryCompilationErrors | ||
import org.apache.spark.sql.execution.QueryExecution | ||
import org.apache.spark.sql.execution.arrow.ArrowConverters | ||
|
@@ -2255,6 +2255,23 @@ class SparkConnectPlanner(val session: SparkSession) { | |
.build() | ||
respBuilder.setExplain(explain) | ||
|
||
case StreamingQueryCommand.CommandCase.EXCEPTION => | ||
val result = query.exception | ||
result.foreach(e => | ||
respBuilder.getExceptionBuilder | ||
.setExceptionMessage(SparkConnectService.extractErrorMessage(e))) | ||
|
||
case StreamingQueryCommand.CommandCase.AWAIT_TERMINATION => | ||
if (command.getAwaitTermination.hasTimeoutMs) { | ||
val terminated = query.awaitTermination(command.getAwaitTermination.getTimeoutMs) | ||
respBuilder.getAwaitTerminationBuilder | ||
.setTerminated(terminated) | ||
} else { | ||
query.awaitTermination() | ||
WweiL marked this conversation as resolved.
Show resolved
Hide resolved
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm .. just to be extra clear, it will be disconnected when it reaches gRPC timeout .. am i correct? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Right this is intended at this stage. @rangadi will push update regarding this I believe There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👌 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The client keep sending heart beat message to keep the RPC connection alive. |
||
respBuilder.getAwaitTerminationBuilder | ||
.setTerminated(true) | ||
} | ||
|
||
case StreamingQueryCommand.CommandCase.COMMAND_NOT_SET => | ||
throw new IllegalArgumentException("Missing command in StreamingQueryCommand") | ||
} | ||
|
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,6 +24,9 @@ | |
from pyspark.sql.streaming.query import ( | ||
StreamingQuery as PySparkStreamingQuery, | ||
) | ||
from pyspark.errors.exceptions.connect import ( | ||
StreamingQueryException as CapturedStreamingQueryException, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why need this? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To align with what is done in sql/streaming/query: https://github.com/apache/spark/blame/master/python/pyspark/sql/streaming/query.py#L25 |
||
) | ||
|
||
__all__ = [ | ||
"StreamingQuery", # TODO(SPARK-43032): "StreamingQueryManager" | ||
|
@@ -66,11 +69,21 @@ def isActive(self) -> bool: | |
|
||
isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__ | ||
|
||
# TODO (SPARK-42960): Implement and uncomment the doc | ||
def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]: | ||
raise NotImplementedError() | ||
cmd = pb2.StreamingQueryCommand() | ||
if timeout is not None: | ||
if not isinstance(timeout, (int, float)) or timeout <= 0: | ||
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) | ||
cmd.await_termination.timeout_ms = int(timeout * 1000) | ||
terminated = self._execute_streaming_query_cmd(cmd).await_termination.terminated | ||
return terminated | ||
else: | ||
await_termination_cmd = pb2.StreamingQueryCommand.AwaitTerminationCommand() | ||
cmd.await_termination.CopyFrom(await_termination_cmd) | ||
self._execute_streaming_query_cmd(cmd) | ||
return None | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
# awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__ | ||
awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__ | ||
|
||
@property | ||
def status(self) -> Dict[str, Any]: | ||
|
@@ -127,9 +140,14 @@ def explain(self, extended: bool = False) -> None: | |
|
||
explain.__doc__ = PySparkStreamingQuery.explain.__doc__ | ||
|
||
# TODO (SPARK-42960): Implement and uncomment the doc | ||
def exception(self) -> Optional[StreamingQueryException]: | ||
raise NotImplementedError() | ||
cmd = pb2.StreamingQueryCommand() | ||
cmd.exception = True | ||
exception = self._execute_streaming_query_cmd(cmd).exception | ||
if exception.HasField("exception_message"): | ||
return CapturedStreamingQueryException(exception.exception_message) | ||
else: | ||
return None | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
exception.__doc__ = PySparkStreamingQuery.exception.__doc__ | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -196,7 +196,7 @@ def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]: | |
>>> sq.stop() | ||
""" | ||
if timeout is not None: | ||
if not isinstance(timeout, (int, float)) or timeout < 0: | ||
if not isinstance(timeout, (int, float)) or timeout <= 0: | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @HyukjinKwon I also change this to be leq to align with StreamExecution's awaitTermination |
||
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) | ||
return self._jsq.awaitTermination(int(timeout * 1000)) | ||
else: | ||
|
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.
We probably need at least the stacktrace. Leave a comment about what our thinking is there.
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.
Yes I agree, currently the support for error is limited.
I tracked how they handle errors, and found that it's through here:
spark/python/pyspark/sql/connect/client.py
Line 1115 in 0c2776a
And then in the
convert_exception
method:spark/python/pyspark/errors/exceptions/connect.py
Lines 57 to 58 in 0c2776a
Only the message is directly passed.
I guess we could file a ticket to wait until batch side's change, and then we could align with them?
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.
I think we should try to be consistent with what current
exception()
returns, which isreturn CapturedStreamingQueryException(msg, stackTrace, je.getCause())
?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.
Right but satckTrace and cause are not included in connect's error framework so far. There is an ongoing PR about this: #40575.