aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2016-04-18 11:09:33 -0700
committerShixiong Zhu <shixiong@databricks.com>2016-04-18 11:09:33 -0700
commit775cf17eaaae1a38efe47b282b1d6bbdb99bd759 (patch)
treeda02930091ff58ad899b8d13f06fc9647fce4b9a /sql/core/src
parent432d1399cb6985893932088875b2f3be981c0b5f (diff)
downloadspark-775cf17eaaae1a38efe47b282b1d6bbdb99bd759.tar.gz
spark-775cf17eaaae1a38efe47b282b1d6bbdb99bd759.tar.bz2
spark-775cf17eaaae1a38efe47b282b1d6bbdb99bd759.zip
[SPARK-14473][SQL] Define analysis rules to catch operations not supported in streaming
## What changes were proposed in this pull request? There are many operations that are currently not supported in the streaming execution. For example: - joining two streams - unioning a stream and a batch source - sorting - window functions (not time windows) - distinct aggregates Furthermore, executing a query with a stream source as a batch query should also fail. This patch add an additional step after analysis in the QueryExecution which will check that all the operations in the analyzed logical plan is supported or not. ## How was this patch tested? unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #12246 from tdas/SPARK-14473.
Diffstat (limited to 'sql/core/src')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala16
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala9
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala11
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala8
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala8
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala34
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala3
10 files changed, 91 insertions, 12 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
index 1343e81569..39d04ed8c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
@@ -20,8 +20,10 @@ package org.apache.spark.sql
import scala.collection.mutable
import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode, UnsupportedOperationChecker}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.util.ContinuousQueryListener
/**
@@ -172,14 +174,23 @@ class ContinuousQueryManager(sqlContext: SQLContext) {
checkpointLocation: String,
df: DataFrame,
sink: Sink,
- trigger: Trigger = ProcessingTime(0)): ContinuousQuery = {
+ trigger: Trigger = ProcessingTime(0),
+ outputMode: OutputMode = Append): ContinuousQuery = {
activeQueriesLock.synchronized {
if (activeQueries.contains(name)) {
throw new IllegalArgumentException(
s"Cannot start query with name $name as a query with that name is already active")
}
+ val analyzedPlan = df.queryExecution.analyzed
+ df.queryExecution.assertAnalyzed()
+
+ if (sqlContext.conf.getConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) {
+ UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
+ }
+
var nextSourceId = 0L
- val logicalPlan = df.logicalPlan.transform {
+
+ val logicalPlan = analyzedPlan.transform {
case StreamingRelation(dataSource, _, output) =>
// Materialize source to avoid creating it in every batch
val metadataPath = s"$checkpointLocation/sources/$nextSourceId"
@@ -195,6 +206,7 @@ class ContinuousQueryManager(sqlContext: SQLContext) {
checkpointLocation,
logicalPlan,
sink,
+ outputMode,
trigger)
query.start()
activeQueries.put(name, query)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index fb3e184a64..1a09d70fb9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -461,9 +461,7 @@ class Dataset[T] private[sql](
* @since 2.0.0
*/
@Experimental
- def isStreaming: Boolean = logicalPlan.find { n =>
- n.isInstanceOf[StreamingRelation] || n.isInstanceOf[StreamingExecutionRelation]
- }.isDefined
+ def isStreaming: Boolean = logicalPlan.isStreaming
/**
* Displays the [[Dataset]] in a tabular form. Strings more than 20 characters will be truncated,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index f5e1e77263..ddcae0fe07 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -20,9 +20,11 @@ package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, SQLContext}
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange}
+import org.apache.spark.sql.internal.SQLConf
/**
* The primary workflow for executing relational queries using Spark. Designed to allow easy
@@ -43,10 +45,17 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) {
throw ae
}
+ def assertSupported(): Unit = {
+ if (sqlContext.conf.getConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) {
+ UnsupportedOperationChecker.checkForBatch(analyzed)
+ }
+ }
+
lazy val analyzed: LogicalPlan = sqlContext.sessionState.analyzer.execute(logical)
lazy val withCachedData: LogicalPlan = {
assertAnalyzed()
+ assertSupported()
sqlContext.cacheManager.useCachedData(analyzed)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
index aaced49dd1..81244ed874 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
@@ -18,9 +18,11 @@
package org.apache.spark.sql.execution.streaming
import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.analysis.{OutputMode, UnsupportedOperationChecker}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryNode}
+import org.apache.spark.sql.internal.SQLConf
/**
* A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]]
@@ -29,6 +31,7 @@ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner,
class IncrementalExecution(
ctx: SQLContext,
logicalPlan: LogicalPlan,
+ outputMode: OutputMode,
checkpointLocation: String,
currentBatchId: Long) extends QueryExecution(ctx, logicalPlan) {
@@ -69,4 +72,7 @@ class IncrementalExecution(
}
override def preparations: Seq[Rule[SparkPlan]] = state +: super.preparations
+
+ /** No need assert supported, as this check has already been done */
+ override def assertSupported(): Unit = { }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index 87dd27a2b1..2a1fa1ba62 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.OutputMode
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
@@ -48,6 +49,7 @@ class StreamExecution(
checkpointRoot: String,
private[sql] val logicalPlan: LogicalPlan,
val sink: Sink,
+ val outputMode: OutputMode,
val trigger: Trigger) extends ContinuousQuery with Logging {
/** An monitor used to wait/notify when batches complete. */
@@ -314,8 +316,13 @@ class StreamExecution(
}
val optimizerStart = System.nanoTime()
- lastExecution =
- new IncrementalExecution(sqlContext, newPlan, checkpointFile("state"), currentBatchId)
+ lastExecution = new IncrementalExecution(
+ sqlContext,
+ newPlan,
+ outputMode,
+ checkpointFile("state"),
+ currentBatchId)
+
lastExecution.executedPlan
val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000
logDebug(s"Optimized batch in ${optimizerTime}ms")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
index d2872e49ce..c29291eb58 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
@@ -37,6 +37,7 @@ object StreamingRelation {
*/
case class StreamingRelation(dataSource: DataSource, sourceName: String, output: Seq[Attribute])
extends LeafNode {
+ override def isStreaming: Boolean = true
override def toString: String = sourceName
}
@@ -45,6 +46,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output:
* [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]].
*/
case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) extends LeafNode {
+ override def isStreaming: Boolean = true
override def toString: String = source.toString
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 70e18cebdd..7f206bdb9b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -442,6 +442,14 @@ object SQLConf {
.stringConf
.createOptional
+ val UNSUPPORTED_OPERATION_CHECK_ENABLED =
+ SQLConfigBuilder("spark.sql.streaming.unsupportedOperationCheck")
+ .internal()
+ .doc("When true, the logical plan for continuous query will be checked for unsupported" +
+ " operations.")
+ .booleanConf
+ .createWithDefault(true)
+
// TODO: This is still WIP and shouldn't be turned on without extensive test coverage
val COLUMNAR_AGGREGATE_MAP_ENABLED = SQLConfigBuilder("spark.sql.codegen.aggregate.map.enabled")
.internal()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala
index 6ccc99fe17..242ea9cb27 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala
@@ -33,6 +33,7 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException
import org.scalatest.time.Span
import org.scalatest.time.SpanSugar._
+import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode}
import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util._
@@ -75,6 +76,8 @@ trait StreamTest extends QueryTest with Timeouts {
/** How long to wait for an active stream to catch up when checking a result. */
val streamingTimeout = 10.seconds
+ val outputMode: OutputMode = Append
+
/** A trait for actions that can be performed while testing a streaming DataFrame. */
trait StreamAction
@@ -228,6 +231,7 @@ trait StreamTest extends QueryTest with Timeouts {
|$testActions
|
|== Stream ==
+ |Output Mode: $outputMode
|Stream state: $currentOffsets
|Thread state: $threadState
|${if (streamDeathCause != null) stackTraceToString(streamDeathCause) else ""}
@@ -235,6 +239,7 @@ trait StreamTest extends QueryTest with Timeouts {
|== Sink ==
|${sink.toDebugString}
|
+ |
|== Plan ==
|${if (currentStream != null) currentStream.lastExecution else ""}
""".stripMargin
@@ -293,7 +298,8 @@ trait StreamTest extends QueryTest with Timeouts {
StreamExecution.nextName,
metadataRoot,
stream,
- sink)
+ sink,
+ outputMode = outputMode)
.asInstanceOf[StreamExecution]
currentStream.microBatchThread.setUncaughtExceptionHandler(
new UncaughtExceptionHandler {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
index 2bd27c7efd..6f3149dbc5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
@@ -17,10 +17,9 @@
package org.apache.spark.sql.streaming
-import org.scalatest.concurrent.Eventually._
-
-import org.apache.spark.sql.{DataFrame, Row, SQLContext, StreamTest}
+import org.apache.spark.sql._
import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.functions._
import org.apache.spark.sql.sources.StreamSourceProvider
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
@@ -108,6 +107,35 @@ class StreamSuite extends StreamTest with SharedSQLContext {
assertDF(df)
assertDF(df)
}
+
+ test("unsupported queries") {
+ val streamInput = MemoryStream[Int]
+ val batchInput = Seq(1, 2, 3).toDS()
+
+ def assertError(expectedMsgs: Seq[String])(body: => Unit): Unit = {
+ val e = intercept[AnalysisException] {
+ body
+ }
+ expectedMsgs.foreach { s => assert(e.getMessage.contains(s)) }
+ }
+
+ // Running streaming plan as a batch query
+ assertError("startStream" :: Nil) {
+ streamInput.toDS.map { i => i }.count()
+ }
+
+ // Running non-streaming plan with as a streaming query
+ assertError("without streaming sources" :: "startStream" :: Nil) {
+ val ds = batchInput.map { i => i }
+ testStream(ds)()
+ }
+
+ // Running streaming plan that cannot be incrementalized
+ assertError("not supported" :: "streaming" :: Nil) {
+ val ds = streamInput.toDS.map { i => i }.sort()
+ testStream(ds)()
+ }
+ }
}
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
index 3af7c01e52..fa3b122f6d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming
import org.apache.spark.SparkException
import org.apache.spark.sql.StreamTest
+import org.apache.spark.sql.catalyst.analysis.Update
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.expressions.scala.typed
import org.apache.spark.sql.functions._
@@ -32,6 +33,8 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext {
import testImplicits._
+ override val outputMode = Update
+
test("simple count") {
val inputData = MemoryStream[Int]