Skip to content

Commit 60bd91a

Browse files
committed
[SPARK-19268][SS] Disallow adaptive query execution for streaming queries
## What changes were proposed in this pull request? As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming. ## How was this patch tested? `test("SPARK-19268: Adaptive query execution should be disallowed")`. Author: Shixiong Zhu <[email protected]> Closes #16683 from zsxwing/SPARK-19268.
1 parent e576c1e commit 60bd91a

File tree

2 files changed

+17
-1
lines changed

2 files changed

+17
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -230,6 +230,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
230230
UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
231231
}
232232

233+
if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) {
234+
throw new AnalysisException(
235+
s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " +
236+
"is not supported in streaming DataFrames/Datasets")
237+
}
238+
233239
new StreamingQueryWrapper(new StreamExecution(
234240
sparkSession,
235241
userSpecifiedName.orNull,

sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,8 +30,9 @@ import org.scalatest.time.Span
3030
import org.scalatest.time.SpanSugar._
3131

3232
import org.apache.spark.SparkException
33-
import org.apache.spark.sql.Dataset
33+
import org.apache.spark.sql.{AnalysisException, Dataset}
3434
import org.apache.spark.sql.execution.streaming._
35+
import org.apache.spark.sql.internal.SQLConf
3536
import org.apache.spark.sql.streaming.util.BlockingSource
3637
import org.apache.spark.util.Utils
3738

@@ -238,6 +239,15 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter {
238239
}
239240
}
240241

242+
test("SPARK-19268: Adaptive query execution should be disallowed") {
243+
withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") {
244+
val e = intercept[AnalysisException] {
245+
MemoryStream[Int].toDS.writeStream.queryName("test-query").format("memory").start()
246+
}
247+
assert(e.getMessage.contains(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) &&
248+
e.getMessage.contains("not supported"))
249+
}
250+
}
241251

242252
/** Run a body of code by defining a query on each dataset */
243253
private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = {

0 commit comments

Comments
 (0)