-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-48567][SS] StreamingQuery.lastProgress should return the actual StreamingQueryProgress #46921
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-48567][SS] StreamingQuery.lastProgress should return the actual StreamingQueryProgress #46921
Changes from 3 commits
c73ebfc
d967119
ef7a116
f8d7403
707d5bd
917adb6
cab14ac
8355342
b43dc96
984d00b
e51e1b1
b2cd7c9
4c8f01e
53afe51
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 | ||
|---|---|---|---|---|
|
|
@@ -394,9 +394,11 @@ def errorClassOnException(self) -> Optional[str]: | |||
| return self._errorClassOnException | ||||
|
|
||||
|
|
||||
| class StreamingQueryProgress: | ||||
| class StreamingQueryProgress(dict): | ||||
| """ | ||||
| .. versionadded:: 3.4.0 | ||||
| .. versionchanged:: 4.0.0 | ||||
| Becomes a subclass of dict | ||||
|
|
||||
| Notes | ||||
| ----- | ||||
|
|
@@ -486,9 +488,9 @@ def fromJson(cls, j: Dict[str, Any]) -> "StreamingQueryProgress": | |||
| stateOperators=[StateOperatorProgress.fromJson(s) for s in j["stateOperators"]], | ||||
| sources=[SourceProgress.fromJson(s) for s in j["sources"]], | ||||
| sink=SinkProgress.fromJson(j["sink"]), | ||||
| numInputRows=j["numInputRows"], | ||||
| inputRowsPerSecond=j["inputRowsPerSecond"], | ||||
| processedRowsPerSecond=j["processedRowsPerSecond"], | ||||
| numInputRows=j["numInputRows"] if "numInputRows" in j else None, | ||||
| inputRowsPerSecond=j["inputRowsPerSecond"] if "inputRowsPerSecond" in j else None, | ||||
| processedRowsPerSecond=j["processedRowsPerSecond"] if "processedRowsPerSecond" in j else None, | ||||
| observedMetrics={ | ||||
| k: Row(*row_dict.keys())(*row_dict.values()) # Assume no nested rows | ||||
| for k, row_dict in j["observedMetrics"].items() | ||||
|
|
@@ -497,6 +499,19 @@ def fromJson(cls, j: Dict[str, Any]) -> "StreamingQueryProgress": | |||
| else {}, | ||||
| ) | ||||
|
|
||||
| def __getitem__(self, key): | ||||
| # Before Spark 4.0, StreamingQuery.lastProgress returns a dict, which casts id and runId | ||||
| # to string. To prevent breaking change, also cast them to string when accessed with | ||||
| # __getitem__. | ||||
| if key == "id" or key == "runId": | ||||
|
||||
| def sources(self) -> List["SourceProgress"]: |
let me also make these subclass of dict...
Outdated
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.
This is in the fear of users ever set the value of the returned dict before this change
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.
Hmmm .. but the end users can't access to this value if I am reading this correctly?
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.
The fear is backward compatibility. This is possible in current master:
>>> q = spark.readStream.format("rate").load().writeStream.format("noop").start()
24/06/10 16:10:35 WARN ResolveWriteToStream: Temporary checkpoint location created which is deleted normally when the query didn't fail: /private/var/folders/9k/pbxb4_690wv4smwhwbzwmqkw0000gp/T/temporary-709975db-23ed-4838-b9ae-93a7ffe59183. If it's required to delete it under any circumstances, please set spark.sql.streaming.forceDeleteTempCheckpointLocation to true. Important to know deleting temp checkpoint folder is best effort.
24/06/10 16:10:35 WARN ResolveWriteToStream: spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets and will be disabled.
>>> p = q.lastProgress
>>> p
{'id': '44510846-29f8-4218-95cf-616efecadb05', 'runId': 'afcac0a7-424b-428b-948e-2c0fc21a43a2', 'name': None, 'timestamp': '2024-06-10T23:10:38.257Z', 'batchId': 2, 'batchDuration': 215, 'numInputRows': 1, 'inputRowsPerSecond': 76.92307692307692, 'processedRowsPerSecond': 4.651162790697675, 'durationMs': {'addBatch': 30, 'commitOffsets': 82, 'getBatch': 0, 'latestOffset': 0, 'queryPlanning': 4, 'triggerExecution': 215, 'walCommit': 98}, 'stateOperators': [], 'sources': [{'description': 'RateStreamV2[rowsPerSecond=1, rampUpTimeSeconds=0, numPartitions=default', 'startOffset': 1, 'endOffset': 2, 'latestOffset': 2, 'numInputRows': 1, 'inputRowsPerSecond': 76.92307692307692, 'processedRowsPerSecond': 4.651162790697675}], 'sink': {'description': 'org.apache.spark.sql.execution.datasources.noop.NoopTable$@67a2b2a4', 'numOutputRows': 1}}
>>> p["id"]
'44510846-29f8-4218-95cf-616efecadb05'
>>> p["id"] = "aaaaaaa"
>>> p["id"]
'aaaaaaa'
This is not possible in Scala of course, but not sure if we should keep this python specific behavior....
Uh oh!
There was an error while loading. Please reload this page.