-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-51358] [SS] Introduce snapshot upload lag detection through StateStoreCoordinator #50123
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 all commits
958b491
7ffadd8
3c6a5f9
6056856
41eaba4
4117326
d039f73
cf6da39
e51b491
6ea790f
f2b84d4
77aa7db
3148211
6ba4dcf
b0c3e81
11b5343
6ed3366
8cb4bbf
8b1fd5b
4ada285
3d79a80
a100f49
2c07bf3
9b7b75e
7a3dca4
ea73d47
3de7008
b46dc63
70b7a8a
a69d44e
29000ec
1723546
9856463
b2a7ccb
ec585a3
f2edfa5
1710fe4
d8b2184
00b01da
735b356
2d60ea9
e4c0cf9
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 |
|---|---|---|
|
|
@@ -36,6 +36,7 @@ import org.apache.spark.sql.connector.catalog.Table | |
| import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReportsSinkMetrics, ReportsSourceMetrics, SparkDataStream} | ||
| import org.apache.spark.sql.execution.{QueryExecution, StreamSourceAwareSparkPlan} | ||
| import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress} | ||
| import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef | ||
| import org.apache.spark.sql.streaming._ | ||
| import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, QueryProgressEvent} | ||
| import org.apache.spark.util.{Clock, Utils} | ||
|
|
@@ -61,6 +62,12 @@ class ProgressReporter( | |
| val noDataProgressEventInterval: Long = | ||
| sparkSession.sessionState.conf.streamingNoDataProgressEventInterval | ||
|
|
||
| val coordinatorReportSnapshotUploadLag: Boolean = | ||
| sparkSession.sessionState.conf.stateStoreCoordinatorReportSnapshotUploadLag | ||
|
|
||
| val stateStoreCoordinator: StateStoreCoordinatorRef = | ||
| sparkSession.sessionState.streamingQueryManager.stateStoreCoordinator | ||
|
|
||
| private val timestampFormat = | ||
| DateTimeFormatter | ||
| .ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 | ||
|
|
@@ -283,6 +290,17 @@ abstract class ProgressContext( | |
| progressReporter.lastNoExecutionProgressEventTime = triggerClock.getTimeMillis() | ||
| progressReporter.updateProgress(newProgress) | ||
|
|
||
| // Ask the state store coordinator to log all lagging state stores | ||
| if (progressReporter.coordinatorReportSnapshotUploadLag) { | ||
|
Contributor
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. nit: should this check to see if it is enabled be done in the
Contributor
Author
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. We do it in both sides right now - this check is added here for both readability and reducing overhead to retrieve the coordinator ref + sending the RPC call |
||
| val latestVersion = lastEpochId + 1 | ||
| progressReporter.stateStoreCoordinator | ||
| .logLaggingStateStores( | ||
| lastExecution.runId, | ||
| latestVersion, | ||
| lastExecution.isTerminatingTrigger | ||
| ) | ||
| } | ||
|
|
||
| // Update the value since this trigger executes a batch successfully. | ||
| this.execStatsOnLatestExecutedBatch = Some(execStats) | ||
|
|
||
|
|
||
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.
Would we also detect the case where its likely that maintenance runs very infrequently ? for eg - if the user is running an availNow query with small batch durations ?
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.
Just added a case for this - thanks!