-
Notifications
You must be signed in to change notification settings - Fork 28k
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-46906][SS] Add a check for stateful operator change for streaming #44927
Changes from 2 commits
eec170c
d2d79a4
98adaaa
89f67ad
37ef7fd
7ecb493
5241dbb
20121cd
e427824
0618e9e
a6619cb
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 |
---|---|---|
|
@@ -29,12 +29,13 @@ import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, Express | |
import org.apache.spark.sql.catalyst.plans.logical._ | ||
import org.apache.spark.sql.catalyst.rules.Rule | ||
import org.apache.spark.sql.catalyst.trees.TreePattern._ | ||
import org.apache.spark.sql.errors.QueryExecutionErrors | ||
import org.apache.spark.sql.execution.{LocalLimitExec, QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} | ||
import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, MergingSessionsExec, ObjectHashAggregateExec, SortAggregateExec, UpdatingSessionsExec} | ||
import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike | ||
import org.apache.spark.sql.execution.python.FlatMapGroupsInPandasWithStateExec | ||
import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSourceV1 | ||
import org.apache.spark.sql.execution.streaming.state.OperatorStateMetadataWriter | ||
import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadataReader, OperatorStateMetadataV1, OperatorStateMetadataWriter} | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.streaming.OutputMode | ||
import org.apache.spark.util.Utils | ||
|
@@ -184,6 +185,41 @@ class IncrementalExecution( | |
} | ||
} | ||
|
||
/** | ||
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. Maybe better to compare in higher level - construct a map opId -> operatorName for both physical plan and state metadata, and compare two. With this approach we can also give a better error message for all cases 1) new addition of stateful operator(s), 2) removal of existing stateful operator(s), 3) replacement of existing stateful operator(s). 3) can also happen if someone tries to replace dropDuplicates with dropDuplicatesWithinWatermark after encountering some issue. Worth noting that we consider state metadata as a source of truth in this PR - so if there is no state metadata prior to this, it would be just same as letting Spark create a new state metadata file and we do comparison (this cannot perform any real check though). So the check can be also done after executing physical planning rules, maybe at the end of 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. Thanks for the review Jungtaek! I also like the idea of adding a map.
I tried to add the check after |
||
* Read from existing operator state metadata that contains the same operator id | ||
* with current operator id. Check if they contains the same operator name. | ||
* Throw errors if not match. | ||
*/ | ||
object checkOperatorInMetadata extends SparkPlanPartialRule { | ||
override val rule: PartialFunction[SparkPlan, SparkPlan] = { | ||
case stateStoreWriter: StateStoreWriter if isFirstBatch => | ||
val opId = stateStoreWriter.getStateInfo.operatorId | ||
try { | ||
val metadataPathToCheck = new Path(checkpointLocation, opId.toString) | ||
logInfo("Reading from operator metadata, check if stateful operator with " + | ||
"the same id from committed batch is the same operator of current stateful operator. " + | ||
s"Stateful operator metadata path to check: ${metadataPathToCheck.toString}") | ||
val operatorMetadata: OperatorStateMetadataV1 = new OperatorStateMetadataReader( | ||
metadataPathToCheck, hadoopConf).read().asInstanceOf[OperatorStateMetadataV1] | ||
val operatorInMetadata = operatorMetadata.operatorInfo.operatorName | ||
if (operatorMetadata.operatorInfo.operatorName != stateStoreWriter.shortName) { | ||
throw QueryExecutionErrors.statefulOperatorNotMatchInStateMetadataError( | ||
opId, stateStoreWriter.shortName, operatorInMetadata) | ||
} | ||
} catch { | ||
case e: java.io.FileNotFoundException => | ||
// no need to throw fatal error | ||
logWarning("Error reading metadata path for stateful operator. " + | ||
"This may due to no prior committed batch, or previously run on lower versions. " + | ||
"Trying to read operator metadata for stateful operator " + | ||
s"$opId: ${e.toString}") | ||
case e: Exception => | ||
throw e | ||
} | ||
stateStoreWriter | ||
} | ||
} | ||
|
||
object WriteStatefulOperatorMetadataRule extends SparkPlanPartialRule { | ||
override val rule: PartialFunction[SparkPlan, SparkPlan] = { | ||
case stateStoreWriter: StateStoreWriter if isFirstBatch => | ||
|
@@ -389,6 +425,9 @@ class IncrementalExecution( | |
|
||
override def apply(plan: SparkPlan): SparkPlan = { | ||
val planWithStateOpId = plan transform composedRule | ||
// Check operator name, fail the query if operator changes; | ||
// Will not change rule | ||
planWithStateOpId transform checkOperatorInMetadata.rule | ||
// The rule doesn't change the plan but cause the side effect that metadata is written | ||
// in the checkpoint directory of stateful operator. | ||
planWithStateOpId transform WriteStatefulOperatorMetadataRule.rule | ||
|
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.
Can we explain why this occur to the customer? Like "changing stateful operator of existing streaming query is not allowed."