-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-53823][SS] Implement allow list for real time mode #52891
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
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 |
|---|---|---|
|
|
@@ -3094,6 +3094,13 @@ object SQLConf { | |
| .timeConf(TimeUnit.MILLISECONDS) | ||
| .createWithDefault(5000) | ||
|
|
||
| val STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK = buildConf( | ||
| "spark.sql.streaming.realTimeMode.allowlistCheck") | ||
| .doc("Whether to check all operators, sinks used in real-time mode are in the allowlist.") | ||
|
Member
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 need a version field. .version("4.1.0") |
||
| .version("4.1.0") | ||
| .booleanConf | ||
| .createWithDefault(true) | ||
|
|
||
| val VARIABLE_SUBSTITUTE_ENABLED = | ||
| buildConf("spark.sql.variable.substitute") | ||
| .doc("This enables substitution using syntax like `${var}`, `${system:var}`, " + | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -853,6 +853,27 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { | |
| Deduplicate(Seq(attribute), streamRelation)), outputMode = Append) | ||
| } | ||
|
|
||
| /* | ||
| ======================================================================================= | ||
| REAL-TIME STREAMING | ||
|
Member
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. BTW, for the actual test methods, it seems that the following |
||
| ======================================================================================= | ||
| */ | ||
|
|
||
| { | ||
| assertNotSupportedForRealTime( | ||
| "real-time without operators - append mode", | ||
| streamRelation, | ||
| Append, | ||
| "STREAMING_REAL_TIME_MODE.OUTPUT_MODE_NOT_SUPPORTED" | ||
| ) | ||
|
|
||
| assertSupportedForRealTime( | ||
| "real-time with stream-batch join - update mode", | ||
| streamRelation.join(batchRelation, joinType = Inner), | ||
| Update | ||
| ) | ||
| } | ||
|
|
||
| /* | ||
| ======================================================================================= | ||
| TESTING FUNCTIONS | ||
|
|
@@ -1017,6 +1038,31 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { | |
| } | ||
| } | ||
|
|
||
| /** Assert that the logical plan is supported for real-time mode */ | ||
| def assertSupportedForRealTime(name: String, plan: LogicalPlan, outputMode: OutputMode): Unit = { | ||
|
Member
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, where is it used? I don't find any.
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. ah forgot to push a commit. The test is added |
||
| test(s"real-time trigger - $name: supported") { | ||
| UnsupportedOperationChecker.checkAdditionalRealTimeModeConstraints(plan, outputMode) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Assert that the logical plan is not supported inside a streaming plan with the | ||
| * real-time trigger. | ||
| */ | ||
| def assertNotSupportedForRealTime( | ||
| name: String, | ||
| plan: LogicalPlan, | ||
| outputMode: OutputMode, | ||
| condition: String): Unit = { | ||
| testError( | ||
| s"real-time trigger - $name: not supported", | ||
| Seq("Streaming real-time mode"), | ||
| condition | ||
| ) { | ||
| UnsupportedOperationChecker.checkAdditionalRealTimeModeConstraints(plan, outputMode) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Assert that the logical plan is not supported inside a streaming plan. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,7 +41,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors | |
| import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} | ||
| import org.apache.spark.sql.execution.datasources.LogicalRelation | ||
| import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, RealTimeStreamScanExec, StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} | ||
| import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, RealTimeTrigger, Sink, Source, StreamingQueryPlanTraverseHelper} | ||
| import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, RealTimeModeAllowlist, RealTimeTrigger, Sink, Source, StreamingQueryPlanTraverseHelper} | ||
| import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitMetadata, OffsetSeq, OffsetSeqMetadata} | ||
| import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOpStateStoreCheckpointInfo, StateStoreWriter} | ||
| import org.apache.spark.sql.execution.streaming.runtime.AcceptsLatestSeenOffsetHandler | ||
|
|
@@ -436,7 +436,10 @@ class MicroBatchExecution( | |
| } | ||
| } | ||
|
|
||
| if (containsStatefulOperator(analyzedPlan)) { | ||
| if (trigger.isInstanceOf[RealTimeTrigger]) { | ||
|
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. @HeartSaVioR you recently added AQE support streaming queries. Can you take a look to see if this logic is appropriate to disable AQE for RTM quieries?
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. That should do the thing. Looks good to me.
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. thx |
||
| logWarning(log"Disabling AQE since AQE is not supported for Real-time Mode.") | ||
| sparkSessionToRunBatches.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") | ||
| } else if (containsStatefulOperator(analyzedPlan)) { | ||
| // SPARK-53941: We disable AQE for stateful workloads as of now. | ||
| logWarning(log"Disabling AQE since AQE is not supported in stateful workloads.") | ||
| sparkSessionToRunBatches.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") | ||
|
|
@@ -1042,6 +1045,14 @@ class MicroBatchExecution( | |
|
|
||
| markMicroBatchExecutionStart(execCtx) | ||
|
|
||
| if (trigger.isInstanceOf[RealTimeTrigger]) { | ||
| RealTimeModeAllowlist.checkAllowedPhysicalOperator( | ||
| execCtx.executionPlan.executedPlan, | ||
| sparkSession.sessionState.conf.getConf( | ||
| SQLConf.STREAMING_REAL_TIME_MODE_ALLOWLIST_CHECK) | ||
| ) | ||
| } | ||
|
|
||
| if (execCtx.previousContext.isEmpty) { | ||
| purgeStatefulMetadataAsync(execCtx.executionPlan.executedPlan) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,148 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution.streaming | ||
|
|
||
| import org.apache.spark.SparkIllegalArgumentException | ||
| import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext} | ||
| import org.apache.spark.sql.connector.catalog.Table | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.datasources.v2.RealTimeStreamScanExec | ||
| import org.apache.spark.sql.execution.streaming.operators.stateful._ | ||
|
|
||
| object RealTimeModeAllowlist extends Logging { | ||
| private val allowedSinks = Set( | ||
| "org.apache.spark.sql.execution.streaming.ConsoleTable$", | ||
| "org.apache.spark.sql.execution.streaming.sources.ContinuousMemorySink", | ||
| "org.apache.spark.sql.execution.streaming.sources.ForeachWriterTable", | ||
| "org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaTable" | ||
| ) | ||
|
|
||
| private val allowedOperators = Set( | ||
| "org.apache.spark.sql.execution.AppendColumnsExec", | ||
| "org.apache.spark.sql.execution.CollectMetricsExec", | ||
| "org.apache.spark.sql.execution.ColumnarToRowExec", | ||
| "org.apache.spark.sql.execution.DeserializeToObjectExec", | ||
| "org.apache.spark.sql.execution.ExpandExec", | ||
| "org.apache.spark.sql.execution.FileSourceScanExec", | ||
| "org.apache.spark.sql.execution.FilterExec", | ||
| "org.apache.spark.sql.execution.GenerateExec", | ||
| "org.apache.spark.sql.execution.InputAdapter", | ||
| "org.apache.spark.sql.execution.LocalTableScanExec", | ||
| "org.apache.spark.sql.execution.MapElementsExec", | ||
| "org.apache.spark.sql.execution.MapPartitionsExec", | ||
| "org.apache.spark.sql.execution.PlanLater", | ||
| "org.apache.spark.sql.execution.ProjectExec", | ||
| "org.apache.spark.sql.execution.RangeExec", | ||
| "org.apache.spark.sql.execution.SerializeFromObjectExec", | ||
| "org.apache.spark.sql.execution.UnionExec", | ||
| "org.apache.spark.sql.execution.WholeStageCodegenExec", | ||
| "org.apache.spark.sql.execution.datasources.v2.RealTimeStreamScanExec", | ||
| "org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec", | ||
| "org.apache.spark.sql.execution.exchange.BroadcastExchangeExec", | ||
| "org.apache.spark.sql.execution.exchange.ReusedExchangeExec", | ||
|
Member
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. I guess this was initially designed to be a sorted list alphabetically in the code side. Could you change like the following because it looks like the only exceptions? |
||
| "org.apache.spark.sql.execution.joins.BroadcastHashJoinExec", | ||
| classOf[EventTimeWatermarkExec].getName | ||
| ) | ||
|
|
||
| private def classNamesString(classNames: Seq[String]): MessageWithContext = { | ||
| val sortedClassNames = classNames.sorted | ||
| var message = log"${MDC(LogKeys.CLASS_NAME, sortedClassNames.head)}" | ||
| sortedClassNames.tail.foreach( | ||
| name => message += log", ${MDC(LogKeys.CLASS_NAME, name)}" | ||
| ) | ||
| if (sortedClassNames.size > 1) { | ||
| message + log" are" | ||
| } else { | ||
| message + log" is" | ||
| } | ||
| } | ||
|
|
||
| private def notInRTMAllowlistException( | ||
| errorType: String, | ||
| classNames: Seq[String]): SparkIllegalArgumentException = { | ||
| assert(classNames.nonEmpty) | ||
| new SparkIllegalArgumentException( | ||
| errorClass = "STREAMING_REAL_TIME_MODE.OPERATOR_OR_SINK_NOT_IN_ALLOWLIST", | ||
| messageParameters = Map( | ||
| "errorType" -> errorType, | ||
| "message" -> classNamesString(classNames).message | ||
| ) | ||
| ) | ||
| } | ||
|
|
||
| def checkAllowedSink(sink: Table, throwException: Boolean): Unit = { | ||
| if (!allowedSinks.contains(sink.getClass.getName)) { | ||
| if (throwException) { | ||
| throw notInRTMAllowlistException("sink", Seq(sink.getClass.getName)) | ||
| } else { | ||
| logWarning( | ||
| log"The sink: " + classNamesString(Seq(sink.getClass.getName)) + | ||
| log" not in the sink allowlist for Real-Time Mode." | ||
| ) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // Collect ALL nodes whose entire subtree contains RealTimeStreamScanExec. | ||
| private def collectRealtimeNodes(root: SparkPlan): Seq[SparkPlan] = { | ||
|
|
||
| def collectNodesWhoseSubtreeHasRTS(n: SparkPlan): (Boolean, List[SparkPlan]) = { | ||
| n match { | ||
| case _: RealTimeStreamScanExec => | ||
| // Subtree has RTS, but we don't collect the RTS node itself. | ||
| (true, Nil) | ||
| case _ if n.children.isEmpty => | ||
| (false, Nil) | ||
| case _ => | ||
| val kidResults = n.children.map(collectNodesWhoseSubtreeHasRTS) | ||
| val anyChildHasRTS = kidResults.exists(_._1) | ||
| val collectedKids = kidResults.iterator.flatMap(_._2).toList | ||
| val collectedHere = if (anyChildHasRTS) n :: collectedKids else collectedKids | ||
| (anyChildHasRTS, collectedHere) | ||
| } | ||
| } | ||
|
|
||
| collectNodesWhoseSubtreeHasRTS(root)._2 | ||
| } | ||
|
|
||
| def checkAllowedPhysicalOperator(operator: SparkPlan, throwException: Boolean): Unit = { | ||
| val nodesToCheck = collectRealtimeNodes(operator) | ||
| val violations = nodesToCheck | ||
| .collect { | ||
| case node => | ||
| if (allowedOperators.contains(node.getClass.getName)) { | ||
| None | ||
| } else { | ||
| Some(node.getClass.getName) | ||
| } | ||
| } | ||
| .flatten | ||
| .distinct | ||
|
|
||
| if (violations.nonEmpty) { | ||
| if (throwException) { | ||
| throw notInRTMAllowlistException("operator", violations.toSet.toSeq) | ||
| } else { | ||
| logWarning( | ||
| log"The operator(s): " + classNamesString(violations.toSet.toSeq) + | ||
| log" not in the operator allowlist for Real-Time Mode." | ||
| ) | ||
| } | ||
| } | ||
| } | ||
| } | ||
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.
nit. Indentation looks a little wrong to me for all realTimeMode configurations, @jerrypeng and @viirya .
It would be great if you can match with the other code around here.