Skip to content

Commit

Permalink
[SPARK-14473][SQL] Define analysis rules to catch operations not supp…
Browse files Browse the repository at this point in the history
…orted 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.
  • Loading branch information
tdas authored and zsxwing committed Apr 18, 2016
1 parent 432d139 commit 775cf17
Show file tree
Hide file tree
Showing 16 changed files with 685 additions and 12 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* 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.catalyst.analysis

sealed trait OutputMode

case object Append extends OutputMode
case object Update extends OutputMode

This comment has been minimized.

Copy link
@lw-lin

lw-lin May 10, 2016

Contributor

These OutputModes as well as their semantics are defined in SPARK-8360. Maybe I'm wrong, but I think there are some mis-matches between what are defined in SPARK-8360 and what we've done.

Given the data:

== batch 1 ===
a, 1
a, 2
== batch 2 ===
a, 3

[1]
For Append, what's the expected output if we run col_1, sum(col_2) group by col_1 on the data?
Should it be this:

== batch 1 ===
a, 3
== batch 2 ===
a, 3

Or should it be this?

== batch 1 ===
a, 3
== batch 2 ===
a, 6

[2]
For Append, what's the expected output if we run select col_1 on the data?
Should it be this:

== batch 1 ===
a
a
== batch 2 ===
a

Or should it be this?

== batch 1 ===
a
a
== batch 2 ===
a
a
a

[3]
For Update, if we run col_1, sum(col_2) group by col_1, the expected sink status should like the following, right?

== updated sink after batch 1 ===
a, 3
== updated sink after batch 2 ===
a, 6

@rxin would you mind clarifying on this when you have time? Thanks!

Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* 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.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._

/**
* Analyzes the presence of unsupported operations in a logical plan.
*/
object UnsupportedOperationChecker {

def checkForBatch(plan: LogicalPlan): Unit = {
plan.foreachUp {
case p if p.isStreaming =>
throwError(
"Queries with streaming sources must be executed with write.startStream()")(p)

case _ =>
}
}

def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = {

if (!plan.isStreaming) {
throwError(
"Queries without streaming sources cannot be executed with write.startStream()")(plan)
}

plan.foreachUp { implicit plan =>

// Operations that cannot exists anywhere in a streaming plan
plan match {

case _: Command =>
throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " +
"streaming DataFrames/Datasets")

case _: InsertIntoTable =>
throwError("InsertIntoTable is not supported with streaming DataFrames/Datasets")

case Aggregate(_, _, child) if child.isStreaming && outputMode == Append =>
throwError(
"Aggregations are not supported on streaming DataFrames/Datasets in " +
"Append output mode. Consider changing output mode to Update.")

case Join(left, right, joinType, _) =>

joinType match {

case Inner =>
if (left.isStreaming && right.isStreaming) {
throwError("Inner join between two streaming DataFrames/Datasets is not supported")
}

case FullOuter =>
if (left.isStreaming || right.isStreaming) {
throwError("Full outer joins with streaming DataFrames/Datasets are not supported")
}


case LeftOuter | LeftSemi | LeftAnti =>
if (right.isStreaming) {
throwError("Left outer/semi/anti joins with a streaming DataFrame/Dataset " +
"on the right is not supported")
}

case RightOuter =>
if (left.isStreaming) {
throwError("Right outer join with a streaming DataFrame/Dataset on the left is " +
"not supported")
}

case NaturalJoin(_) | UsingJoin(_, _) =>
// They should not appear in an analyzed plan.

case _ =>
throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset")
}

case c: CoGroup if c.children.exists(_.isStreaming) =>
throwError("CoGrouping with a streaming DataFrame/Dataset is not supported")

case u: Union if u.children.map(_.isStreaming).distinct.size == 2 =>
throwError("Union between streaming and batch DataFrames/Datasets is not supported")

case Except(left, right) if right.isStreaming =>
throwError("Except with a streaming DataFrame/Dataset on the right is not supported")

case Intersect(left, right) if left.isStreaming && right.isStreaming =>
throwError("Intersect between two streaming DataFrames/Datasets is not supported")

case GroupingSets(_, _, child, _) if child.isStreaming =>
throwError("GroupingSets is not supported on streaming DataFrames/Datasets")

case GlobalLimit(_, _) | LocalLimit(_, _) if plan.children.forall(_.isStreaming) =>
throwError("Limits are not supported on streaming DataFrames/Datasets")

case Sort(_, _, _) | SortPartitions(_, _) if plan.children.forall(_.isStreaming) =>
throwError("Sorting is not supported on streaming DataFrames/Datasets")

case Sample(_, _, _, _, child) if child.isStreaming =>
throwError("Sampling is not supported on streaming DataFrames/Datasets")

case Window(_, _, _, child) if child.isStreaming =>
throwError("Non-time-based windows are not supported on streaming DataFrames/Datasets")

case ReturnAnswer(child) if child.isStreaming =>
throwError("Cannot return immediate result on streaming DataFrames/Dataset. Queries " +
"with streaming DataFrames/Datasets must be executed with write.startStream().")

case _ =>
}
}
}

private def throwErrorIf(
condition: Boolean,
msg: String)(implicit operator: LogicalPlan): Unit = {
if (condition) {
throwError(msg)
}
}

private def throwError(msg: String)(implicit operator: LogicalPlan): Nothing = {
throw new AnalysisException(
msg, operator.origin.line, operator.origin.startPosition, Some(operator))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,24 @@ package object dsl {
condition: Option[Expression] = None): LogicalPlan =
Join(logicalPlan, otherPlan, joinType, condition)

def cogroup[Key: Encoder, Left: Encoder, Right: Encoder, Result: Encoder](
otherPlan: LogicalPlan,
func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result],
leftGroup: Seq[Attribute],
rightGroup: Seq[Attribute],
leftAttr: Seq[Attribute],
rightAttr: Seq[Attribute]
): LogicalPlan = {
CoGroup.apply[Key, Left, Right, Result](
func,
leftGroup,
rightGroup,
leftAttr,
rightAttr,
logicalPlan,
otherPlan)
}

def orderBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, true, logicalPlan)

def sortBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, false, logicalPlan)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
*/
def analyzed: Boolean = _analyzed

/** Returns true if this subtree contains any streaming data sources. */
def isStreaming: Boolean = children.exists(_.isStreaming == true)

/**
* Returns a copy of this node where `rule` has been recursively applied first to all of its
* children and then itself (post-order). When `rule` does not apply to a given node, it is left
Expand Down

0 comments on commit 775cf17

Please sign in to comment.