Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12246#discussion_r59936839
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
 ---
    @@ -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 plan.children.exists(_.isStreaming) =>
    +          throwError("CoGrouping between two streaming DataFrames/Datasets 
is not supported")
    +
    +        case u: Union if u.children.count(_.isStreaming) == 1 =>
    --- End diff --
    
    Correction. I didnt realize that union can have multiple children, not just 
two. Fixed the bug.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to