Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) {
UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
}

if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) {
throw new AnalysisException(
s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " +
"is not supported in streaming DataFrames/Datasets")
}

new StreamingQueryWrapper(new StreamExecution(
sparkSession,
userSpecifiedName.orNull,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,9 @@ import org.scalatest.time.Span
import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkException
import org.apache.spark.sql.Dataset
import org.apache.spark.sql.{AnalysisException, Dataset}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.util.BlockingSource
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -238,6 +239,15 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter {
}
}

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

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